You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by vi...@apache.org on 2016/10/28 18:03:59 UTC

[01/50] [abbrv] hadoop git commit: MAPREDUCE-6541. Exclude scheduled reducer memory when calculating available mapper slots from headroom to avoid deadlock. Contributed by Varun Saxena

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-9806 c37346d0e -> 1eae719bc


MAPREDUCE-6541. Exclude scheduled reducer memory when calculating available mapper slots from headroom to avoid deadlock. Contributed by Varun Saxena


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

Branch: refs/heads/HDFS-9806
Commit: 060558c6f221ded0b014189d5b82eee4cc7b576b
Parents: 8bd6d77
Author: Naganarasimha <na...@apache.org>
Authored: Thu Oct 27 18:03:13 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Thu Oct 27 18:03:13 2016 +0530

----------------------------------------------------------------------
 .../v2/app/rm/RMContainerAllocator.java         |  24 ++--
 .../v2/app/rm/TestRMContainerAllocator.java     | 122 +++++++++++++++++++
 2 files changed, 137 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/060558c6/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index 4cb3cbe..e3b673a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -338,6 +338,12 @@ public class RMContainerAllocator extends RMContainerRequestor
     return scheduledRequests;
   }
 
+  @Private
+  @VisibleForTesting
+  int getNumOfPendingReduces() {
+    return pendingReduces.size();
+  }
+
   public boolean getIsReduceStarted() {
     return reduceStarted;
   }
@@ -521,15 +527,20 @@ public class RMContainerAllocator extends RMContainerRequestor
     }
 
     // The pending mappers haven't been waiting for too long. Let us see if
-    // the headroom can fit a mapper.
-    Resource availableResourceForMap = getAvailableResources();
+    // there are enough resources for a mapper to run. This is calculated by
+    // excluding scheduled reducers from headroom and comparing it against
+    // resources required to run one mapper.
+    Resource scheduledReducesResource = Resources.multiply(
+         reduceResourceRequest, scheduledRequests.reduces.size());
+    Resource availableResourceForMap =
+         Resources.subtract(getAvailableResources(), scheduledReducesResource);
     if (ResourceCalculatorUtils.computeAvailableContainers(availableResourceForMap,
         mapResourceRequest, getSchedulerResourceTypes()) > 0) {
-      // the available headroom is enough to run a mapper
+       // Enough room to run a mapper
       return false;
     }
 
-    // Available headroom is not enough to run mapper. See if we should hold
+    // Available resources are not enough to run mapper. See if we should hold
     // off before preempting reducers and preempt if okay.
     return preemptReducersForHangingMapRequests(reducerNoHeadroomPreemptionDelayMs);
   }
@@ -990,11 +1001,6 @@ public class RMContainerAllocator extends RMContainerRequestor
       Resources.add(assignedMapResource, assignedReduceResource));
   }
 
-  @VisibleForTesting
-  public int getNumOfPendingReduces() {
-    return pendingReduces.size();
-  }
-
   @Private
   @VisibleForTesting
   class ScheduledRequests {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/060558c6/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index 38a9731..bcce793 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -3190,6 +3190,128 @@ public class TestRMContainerAllocator {
     }
   }
 
+  /**
+   * Tests whether scheduled reducers are excluded from headroom while
+   * calculating headroom.
+   */
+  @Test
+  public void testExcludeSchedReducesFromHeadroom() throws Exception {
+    LOG.info("Running testExcludeSchedReducesFromHeadroom");
+    Configuration conf = new Configuration();
+    conf.setInt(MRJobConfig.MR_JOB_REDUCER_UNCONDITIONAL_PREEMPT_DELAY_SEC, -1);
+    MyResourceManager rm = new MyResourceManager(conf);
+    rm.start();
+    DrainDispatcher dispatcher =
+        (DrainDispatcher) rm.getRMContext().getDispatcher();
+
+    // Submit the application
+    RMApp app = rm.submitApp(1024);
+    dispatcher.await();
+
+    MockNM amNodeManager = rm.registerNode("amNM:1234", 1260);
+    amNodeManager.nodeHeartbeat(true);
+    dispatcher.await();
+
+    ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
+        .getAppAttemptId();
+    rm.sendAMLaunched(appAttemptId);
+    dispatcher.await();
+
+    JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0);
+    Job mockJob = mock(Job.class);
+    when(mockJob.getReport()).thenReturn(
+        MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
+            0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
+    Task mockTask = mock(Task.class);
+    TaskAttempt mockTaskAttempt = mock(TaskAttempt.class);
+    when(mockJob.getTask((TaskId)any())).thenReturn(mockTask);
+    when(mockTask.getAttempt((TaskAttemptId)any())).thenReturn(mockTaskAttempt);
+    when(mockTaskAttempt.getProgress()).thenReturn(0.01f);
+    MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
+        appAttemptId, mockJob);
+
+    MockNM nodeManager = rm.registerNode("h1:1234", 4096);
+    dispatcher.await();
+    // Register nodes to RM.
+    MockNM nodeManager2 = rm.registerNode("h2:1234", 1024);
+    dispatcher.await();
+
+    // Request 2 maps and 1 reducer(sone on nodes which are not registered).
+    ContainerRequestEvent event1 =
+        createReq(jobId, 1, 1024, new String[] { "h1" });
+    allocator.sendRequest(event1);
+    ContainerRequestEvent event2 =
+        createReq(jobId, 2, 1024, new String[] { "h2" });
+    allocator.sendRequest(event2);
+    ContainerRequestEvent event3 =
+         createReq(jobId, 3, 1024, new String[] { "h1" }, false, true);
+    allocator.sendRequest(event3);
+
+    // This will tell the scheduler about the requests but there will be no
+    // allocations as nodes are not added.
+    allocator.schedule();
+    dispatcher.await();
+
+    // Request for another reducer on h3 which has not registered.
+    ContainerRequestEvent event4 =
+        createReq(jobId, 4, 1024, new String[] { "h3" }, false, true);
+    allocator.sendRequest(event4);
+
+    allocator.schedule();
+    dispatcher.await();
+
+    // Update resources in scheduler through node heartbeat from h1.
+    nodeManager.nodeHeartbeat(true);
+    dispatcher.await();
+
+    rm.getMyFifoScheduler().forceResourceLimit(Resource.newInstance(3072, 3));
+    allocator.schedule();
+    dispatcher.await();
+
+    // Two maps are assigned.
+    Assert.assertEquals(2, allocator.getAssignedRequests().maps.size());
+    // Send deallocate request for map so that no maps are assigned after this.
+    ContainerAllocatorEvent deallocate1 = createDeallocateEvent(jobId, 1, false);
+    allocator.sendDeallocate(deallocate1);
+    ContainerAllocatorEvent deallocate2 = createDeallocateEvent(jobId, 2, false);
+    allocator.sendDeallocate(deallocate2);
+    // No map should be assigned.
+    Assert.assertEquals(0, allocator.getAssignedRequests().maps.size());
+
+    nodeManager.nodeHeartbeat(true);
+    dispatcher.await();
+
+    rm.getMyFifoScheduler().forceResourceLimit(Resource.newInstance(1024, 1));
+    allocator.schedule();
+    dispatcher.await();
+
+    // h2 heartbeats.
+    nodeManager2.nodeHeartbeat(true);
+    dispatcher.await();
+
+    // Send request for one more mapper.
+    ContainerRequestEvent event5 =
+        createReq(jobId, 5, 1024, new String[] { "h1" });
+    allocator.sendRequest(event5);
+
+    rm.getMyFifoScheduler().forceResourceLimit(Resource.newInstance(2048, 2));
+    allocator.schedule();
+    dispatcher.await();
+    // One reducer is assigned and one map is scheduled
+    Assert.assertEquals(1, allocator.getScheduledRequests().maps.size());
+    Assert.assertEquals(1, allocator.getAssignedRequests().reduces.size());
+    // Headroom enough to run a mapper if headroom is taken as it is but wont be
+    // enough if scheduled reducers resources are deducted.
+    rm.getMyFifoScheduler().forceResourceLimit(Resource.newInstance(1260, 2));
+    allocator.schedule();
+    dispatcher.await();
+    // After allocate response, the one assigned reducer is preempted and killed
+    Assert.assertEquals(1, MyContainerAllocator.getTaskAttemptKillEvents().size());
+    Assert.assertEquals(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC,
+        MyContainerAllocator.getTaskAttemptKillEvents().get(0).getMessage());
+    Assert.assertEquals(1, allocator.getNumOfPendingReduces());
+  }
+
   private static class MockScheduler implements ApplicationMasterProtocol {
     ApplicationAttemptId attemptId;
     long nextContainerId = 10;


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


[45/50] [abbrv] hadoop git commit: HDFS-10926. Update staled configuration properties related to erasure coding. Contributed by Sammi Chen

Posted by vi...@apache.org.
HDFS-10926. Update staled configuration properties related to erasure coding. Contributed by Sammi Chen


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

Branch: refs/heads/HDFS-9806
Commit: 8fa219569762048345d69d15e325c0917361ff66
Parents: 57187fd
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Oct 28 12:37:30 2016 +0600
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Oct 28 12:37:30 2016 +0600

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md        | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fa21956/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 9119b1e..dbd1f44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -106,9 +106,10 @@ Deployment
 
   Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters:
 
-  1. `dfs.datanode.stripedread.timeout.millis` - Timeout for striped reads. Default value is 5000 ms.
-  1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads.
-  1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB.
+  1. `dfs.datanode.ec.reconstruction.stripedread.timeout.millis` - Timeout for striped reads. Default value is 5000 ms.
+  1. `dfs.datanode.ec.reconstruction.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads.
+  1. `dfs.datanode.ec.reconstruction.stripedread.buffer.size` - Buffer size for reader service. Default value is 64KB.
+  1. `dfs.datanode.ec.reconstruction.stripedblock.threads.size` - Number of threads used by the Datanode for background reconstruction work. Default value is 8 threads.
 
 ### Enable Intel ISA-L
 


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


[42/50] [abbrv] hadoop git commit: YARN-4743. FairSharePolicy breaks TimSort assumption. (Zephyr Guo and Yufei Gu via kasha)

Posted by vi...@apache.org.
YARN-4743. FairSharePolicy breaks TimSort assumption. (Zephyr Guo and Yufei Gu via kasha)


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

Branch: refs/heads/HDFS-9806
Commit: 4df8ed63ed93f2542e4b48f521b0cc6624ab59c1
Parents: b2c4f24
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Thu Oct 27 17:42:44 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Thu Oct 27 17:45:48 2016 -0700

----------------------------------------------------------------------
 .../fair/policies/FairSharePolicy.java          |  31 ++-
 .../scheduler/fair/TestSchedulingPolicy.java    | 228 +++++++++++++++++++
 2 files changed, 254 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df8ed63/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 6aa8405..f120f0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -63,7 +63,11 @@ public class FairSharePolicy extends SchedulingPolicy {
    * 
    * Schedulables above their min share are compared by (runningTasks / weight).
    * If all weights are equal, slots are given to the job with the fewest tasks;
-   * otherwise, jobs with more weight get proportionally more slots.
+   * otherwise, jobs with more weight get proportionally more slots. If weight
+   * equals to 0, we can't compare Schedulables by (resource usage/weight).
+   * There are two situations: 1)All weights equal to 0, slots are given
+   * to one with less resource usage. 2)Only one of weight equals to 0, slots
+   * are given to the one with non-zero weight.
    */
   private static class FairShareComparator implements Comparator<Schedulable>,
       Serializable {
@@ -74,6 +78,7 @@ public class FairSharePolicy extends SchedulingPolicy {
     public int compare(Schedulable s1, Schedulable s2) {
       double minShareRatio1, minShareRatio2;
       double useToWeightRatio1, useToWeightRatio2;
+      double weight1, weight2;
       Resource minShare1 = Resources.min(RESOURCE_CALCULATOR, null,
           s1.getMinShare(), s1.getDemand());
       Resource minShare2 = Resources.min(RESOURCE_CALCULATOR, null,
@@ -86,10 +91,26 @@ public class FairSharePolicy extends SchedulingPolicy {
           / Resources.max(RESOURCE_CALCULATOR, null, minShare1, ONE).getMemorySize();
       minShareRatio2 = (double) s2.getResourceUsage().getMemorySize()
           / Resources.max(RESOURCE_CALCULATOR, null, minShare2, ONE).getMemorySize();
-      useToWeightRatio1 = s1.getResourceUsage().getMemorySize() /
-          s1.getWeights().getWeight(ResourceType.MEMORY);
-      useToWeightRatio2 = s2.getResourceUsage().getMemorySize() /
-          s2.getWeights().getWeight(ResourceType.MEMORY);
+
+      weight1 = s1.getWeights().getWeight(ResourceType.MEMORY);
+      weight2 = s2.getWeights().getWeight(ResourceType.MEMORY);
+      if (weight1 > 0.0 && weight2 > 0.0) {
+        useToWeightRatio1 = s1.getResourceUsage().getMemorySize() / weight1;
+        useToWeightRatio2 = s2.getResourceUsage().getMemorySize() / weight2;
+      } else { // Either weight1 or weight2 equals to 0
+        if (weight1 == weight2) {
+          // If they have same weight, just compare usage
+          useToWeightRatio1 = s1.getResourceUsage().getMemorySize();
+          useToWeightRatio2 = s2.getResourceUsage().getMemorySize();
+        } else {
+          // By setting useToWeightRatios to negative weights, we give the
+          // zero-weight one less priority, so the non-zero weight one will
+          // be given slots.
+          useToWeightRatio1 = -weight1;
+          useToWeightRatio2 = -weight2;
+        }
+      }
+
       int res = 0;
       if (s1Needy && !s2Needy)
         res = -1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df8ed63/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
index eeedb09..dea2dd1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
@@ -21,13 +21,25 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 public class TestSchedulingPolicy {
+  private static final Log LOG = LogFactory.getLog(TestSchedulingPolicy.class);
 
   @Test(timeout = 1000)
   public void testParseSchedulingPolicy()
@@ -125,4 +137,220 @@ public class TestSchedulingPolicy {
     assertFalse(ERR,
         SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ANY));
   }
+
+  /**
+   * Test whether {@link FairSharePolicy.FairShareComparator} is transitive.
+   */
+  @Test
+  public void testFairShareComparatorTransitivity() {
+    FairSharePolicy policy = new FairSharePolicy();
+    Comparator<Schedulable> fairShareComparator = policy.getComparator();
+    FairShareComparatorTester tester =
+        new FairShareComparatorTester(fairShareComparator);
+    tester.testTransitivity();
+  }
+
+
+  /**
+   * This class is responsible for testing the transitivity of
+   * {@link FairSharePolicy.FairShareComparator}. We will generate
+   * a lot of triples(each triple contains three {@link Schedulable}),
+   * and then we verify transitivity by using each triple.
+   *
+   * <p>How to generate:</p>
+   * For each field in {@link Schedulable} we all have a data collection. We
+   * combine these data to construct a {@link Schedulable}, and generate all
+   * cases of triple by DFS(depth first search algorithm). We can get 100% code
+   * coverage by DFS.
+   */
+  private class FairShareComparatorTester {
+    private Comparator<Schedulable> fairShareComparator;
+
+    // Use the following data collections to generate three Schedulable.
+    private Resource minShare = Resource.newInstance(0, 1);
+
+    private Resource demand = Resource.newInstance(4, 1);
+
+    private String[] nameCollection = {"A", "B", "C"};
+
+    private long[] startTimeColloection = {1L, 2L, 3L};
+
+    private Resource[] usageCollection = {
+        Resource.newInstance(0, 1), Resource.newInstance(2, 1),
+        Resource.newInstance(4, 1) };
+
+    private ResourceWeights[] weightsCollection = {
+        new ResourceWeights(0.0f), new ResourceWeights(1.0f),
+        new ResourceWeights(2.0f) };
+
+
+
+    public FairShareComparatorTester(
+        Comparator<Schedulable> fairShareComparator) {
+      this.fairShareComparator = fairShareComparator;
+    }
+
+    public void testTransitivity() {
+      generateAndTest(new Stack<Schedulable>());
+    }
+
+    private void generateAndTest(Stack<Schedulable> genSchedulable) {
+      if (genSchedulable.size() == 3) {
+        // We get three Schedulable objects, let's use them to check the
+        // comparator.
+        Assert.assertTrue("The comparator must ensure transitivity",
+            checkTransitivity(genSchedulable));
+        return;
+      }
+
+      for (int i = 0; i < nameCollection.length; i++) {
+        for (int j = 0; j < startTimeColloection.length; j++) {
+          for (int k = 0; k < usageCollection.length; k++) {
+            for (int t = 0; t < weightsCollection.length; t++) {
+              genSchedulable.push(createSchedulable(i, j, k, t));
+              generateAndTest(genSchedulable);
+              genSchedulable.pop();
+            }
+          }
+        }
+      }
+
+    }
+
+    private Schedulable createSchedulable(
+        int nameIdx, int startTimeIdx, int usageIdx, int weightsIdx) {
+      return new MockSchedulable(minShare, demand, nameCollection[nameIdx],
+        startTimeColloection[startTimeIdx], usageCollection[usageIdx],
+        weightsCollection[weightsIdx]);
+    }
+
+    private boolean checkTransitivity(
+        Collection<Schedulable> schedulableObjs) {
+
+      Assert.assertEquals(3, schedulableObjs.size());
+      Schedulable[] copy = schedulableObjs.toArray(new Schedulable[3]);
+
+      if (fairShareComparator.compare(copy[0], copy[1]) > 0) {
+        swap(copy, 0, 1);
+      }
+
+      if (fairShareComparator.compare(copy[1], copy[2]) > 0) {
+        swap(copy, 1, 2);
+
+        if (fairShareComparator.compare(copy[0], copy[1]) > 0) {
+          swap(copy, 0, 1);
+        }
+      }
+
+      // Here, we have got the following condition:
+      // copy[0] <= copy[1] && copy[1] <= copy[2]
+      //
+      // So, just check copy[0] <= copy[2]
+      if (fairShareComparator.compare(copy[0], copy[2]) <= 0) {
+        return true;
+      } else {
+        LOG.fatal("Failure data: " + copy[0] + " " + copy[1] + " " + copy[2]);
+        return false;
+      }
+    }
+
+    private void swap(Schedulable[] array, int x, int y) {
+      Schedulable tmp = array[x];
+      array[x] = array[y];
+      array[y] = tmp;
+    }
+
+
+    private class MockSchedulable implements Schedulable {
+      private Resource minShare;
+      private Resource demand;
+      private String name;
+      private long startTime;
+      private Resource usage;
+      private ResourceWeights weights;
+
+      public MockSchedulable(Resource minShare, Resource demand, String name,
+          long startTime, Resource usage, ResourceWeights weights) {
+        this.minShare = minShare;
+        this.demand = demand;
+        this.name = name;
+        this.startTime = startTime;
+        this.usage = usage;
+        this.weights = weights;
+      }
+
+      @Override
+      public String getName() {
+        return name;
+      }
+
+      @Override
+      public Resource getDemand() {
+        return demand;
+      }
+
+      @Override
+      public Resource getResourceUsage() {
+        return usage;
+      }
+
+      @Override
+      public Resource getMinShare() {
+        return minShare;
+      }
+
+      @Override
+      public ResourceWeights getWeights() {
+        return weights;
+      }
+
+      @Override
+      public long getStartTime() {
+        return startTime;
+      }
+
+      @Override
+      public Resource getMaxShare() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public Priority getPriority() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void updateDemand() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public Resource assignContainer(FSSchedulerNode node) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public RMContainer preemptContainer() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public Resource getFairShare() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void setFairShare(Resource fairShare) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public String toString() {
+        return "{name:" + name + ", start:" + startTime + ", usage:" + usage +
+            ", weights:" + weights + ", demand:" + demand +
+            ", minShare:" + minShare + "}";
+      }
+    }
+  }
+
 }


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


[03/50] [abbrv] hadoop git commit: Revert "Addendum patch for HADOOP-13514 Upgrade maven surefire plugin to 2.19.1. Contributed by Akira Ajisaka."

Posted by vi...@apache.org.
Revert "Addendum patch for HADOOP-13514 Upgrade maven surefire plugin to 2.19.1. Contributed by Akira Ajisaka."

This reverts commit e48b592f8ba1d8a89587f2c4403d861f2d015a9a.


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

Branch: refs/heads/HDFS-9806
Commit: 94e77e9115167e41cd9897472159b1eda24230ab
Parents: b3c15e4
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 27 15:18:24 2016 +0200
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Oct 27 15:18:24 2016 +0200

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94e77e91/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 7537a83..c7c5a72 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1372,7 +1372,7 @@
             <test.build.data>${test.build.data}</test.build.data>
             <test.build.webapps>${test.build.webapps}</test.build.webapps>
             <test.cache.data>${test.cache.data}</test.cache.data>
-            <test.build.classes>${project.build.directory}/test-classes</test.build.classes>
+            <test.build.classes>${test.build.classes}</test.build.classes>
 
             <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
             <java.security.krb5.conf>${project.build.directory}/test-classes/krb5.conf</java.security.krb5.conf>


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


[37/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 cbf8a55..cbdbeaa 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
@@ -34,6 +34,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -127,7 +128,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("attempts")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         verifyAMTaskAttempts(json, task);
       }
@@ -146,7 +148,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("attempts/")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         verifyAMTaskAttempts(json, task);
       }
@@ -165,7 +168,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("attempts")
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         verifyAMTaskAttempts(json, task);
       }
@@ -185,7 +189,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
             .path("jobs").path(jobId).path("tasks").path(tid).path("attempts")
             .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-        assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         String xml = response.getEntity(String.class);
         DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
         DocumentBuilder db = dbf.newDocumentBuilder();
@@ -220,7 +225,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
               .path("jobs").path(jobId).path("tasks").path(tid)
               .path("attempts").path(attid).accept(MediaType.APPLICATION_JSON)
               .get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("taskAttempt");
@@ -249,7 +255,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
               .path("jobs").path(jobId).path("tasks").path(tid)
               .path("attempts").path(attid + "/")
               .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("taskAttempt");
@@ -277,7 +284,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           ClientResponse response = r.path("ws").path("v1").path("mapreduce")
               .path("jobs").path(jobId).path("tasks").path(tid)
               .path("attempts").path(attid).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("taskAttempt");
@@ -305,7 +313,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
               .path("attempts").path(attid).accept(MediaType.APPLICATION_XML)
               .get(ClientResponse.class);
 
-          assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+              response.getType().toString());
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
           DocumentBuilder db = dbf.newDocumentBuilder();
@@ -377,7 +386,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
         } catch (UniformInterfaceException ue) {
           ClientResponse response = ue.getResponse();
           assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject msg = response.getEntity(JSONObject.class);
           JSONObject exception = msg.getJSONObject("RemoteException");
           assertEquals("incorrect number of elements", 3, exception.length());
@@ -556,7 +566,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
               .path("jobs").path(jobId).path("tasks").path(tid)
               .path("attempts").path(attid).path("counters")
               .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("jobTaskAttemptCounters");
@@ -584,7 +595,8 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
               .path("attempts").path(attid).path("counters")
               .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-          assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+              response.getType().toString());
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
           DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 cf71b34..ee7bb0e 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -156,7 +157,8 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("conf")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("conf");
@@ -174,7 +176,8 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("conf/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("conf");
@@ -191,7 +194,8 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
 
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("conf").get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("conf");
@@ -209,7 +213,8 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("conf")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 a38d630..cc57134 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
@@ -34,6 +34,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -123,7 +124,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("jobs").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -140,7 +142,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("jobs/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -156,7 +159,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("jobs").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -173,7 +177,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("jobs").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -198,7 +203,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("job");
@@ -217,7 +223,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId + "/").accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("job");
@@ -234,7 +241,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("job");
@@ -254,7 +262,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -281,7 +290,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -304,7 +314,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -327,7 +338,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String msg = response.getEntity(String.class);
       System.out.println(msg);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -366,7 +378,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -395,7 +408,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).accept(MediaType.APPLICATION_XML)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -629,7 +643,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("counters")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -647,7 +662,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("counters/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -664,7 +680,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("counters/").get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -682,7 +699,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("counters")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -781,7 +799,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobAttempts");
@@ -799,7 +818,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobAttempts");
@@ -817,7 +837,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts")
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobAttempts");
@@ -835,7 +856,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 242c243..ab4d818 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
@@ -32,6 +32,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
@@ -122,7 +123,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("tasks")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -141,7 +143,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       String jobId = MRApps.toString(id);
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("tasks").get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -161,7 +164,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("tasks/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -182,7 +186,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("tasks")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -206,7 +211,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("tasks").queryParam("type", type)
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -226,7 +232,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("mapreduce")
           .path("jobs").path(jobId).path("tasks").queryParam("type", type)
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -253,7 +260,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -282,7 +290,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("task");
@@ -303,7 +312,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid + "/")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("task");
@@ -324,7 +334,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("task");
@@ -347,7 +358,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -381,7 +393,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -413,7 +426,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -447,7 +461,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -481,7 +496,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -514,7 +530,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             .path("jobs").path(jobId).path("tasks").path(tid)
             .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-        assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         String xml = response.getEntity(String.class);
         DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
         DocumentBuilder db = dbf.newDocumentBuilder();
@@ -627,7 +644,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("counters")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("jobTaskCounters");
@@ -648,7 +666,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("counters/")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("jobTaskCounters");
@@ -669,7 +688,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("counters")
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("jobTaskCounters");
@@ -690,7 +710,8 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("mapreduce")
             .path("jobs").path(jobId).path("tasks").path(tid).path("counters")
             .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         String xml = response.getEntity(String.class);
         DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
         DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
index c95ce3e..dda237d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
@@ -34,6 +34,7 @@ import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.UriInfo;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -110,14 +111,16 @@ public class HsWebServices {
   }
 
   @GET
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public HistoryInfo get() {
     return getHistoryInfo();
   }
 
   @GET
   @Path("/info")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public HistoryInfo getHistoryInfo() {
     init();
     return new HistoryInfo();
@@ -125,7 +128,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobsInfo getJobs(@QueryParam("user") String userQuery,
       @QueryParam("limit") String count,
       @QueryParam("state") String stateQuery,
@@ -215,7 +219,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobInfo getJob(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid) {
 
@@ -227,7 +232,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/jobattempts")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AMAttemptsInfo getJobAttempts(@PathParam("jobid") String jid) {
 
     init();
@@ -244,7 +250,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/counters")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobCounterInfo getJobCounters(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid) {
 
@@ -256,7 +263,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/conf")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ConfInfo getJobConf(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid) {
 
@@ -275,7 +283,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/tasks")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TasksInfo getJobTasks(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @QueryParam("type") String type) {
 
@@ -302,7 +311,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TaskInfo getJobTask(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
 
@@ -316,7 +326,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/counters")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobTaskCounterInfo getSingleTaskCounters(
       @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
       @PathParam("taskid") String tid) {
@@ -337,7 +348,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TaskAttemptsInfo getJobTaskAttempts(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
 
@@ -360,7 +372,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TaskAttemptInfo getJobTaskAttemptId(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid,
       @PathParam("attemptid") String attId) {
@@ -380,7 +393,8 @@ public class HsWebServices {
 
   @GET
   @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
       @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
       @PathParam("taskid") String tid, @PathParam("attemptid") String attId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 ab812bb..64fd2ca 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
@@ -29,6 +29,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.hs.HistoryContext;
 import org.apache.hadoop.mapreduce.v2.hs.JobHistory;
@@ -116,7 +117,8 @@ public class TestHsWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("history")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
@@ -127,7 +129,8 @@ public class TestHsWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("history/")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
@@ -138,7 +141,8 @@ public class TestHsWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("history/")
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
@@ -149,7 +153,8 @@ public class TestHsWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("history")
         .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyHSInfoXML(xml, appContext);
   }
@@ -160,7 +165,8 @@ public class TestHsWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("info").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
@@ -172,7 +178,8 @@ public class TestHsWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("info/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
@@ -183,7 +190,8 @@ public class TestHsWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("info/").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
@@ -195,7 +203,8 @@ public class TestHsWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("info/").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyHSInfoXML(xml, appContext);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 38058b6..ddbe23f 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
@@ -35,6 +35,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -137,7 +138,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("attempts").accept(MediaType.APPLICATION_JSON)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         verifyHsTaskAttempts(json, task);
       }
@@ -157,7 +159,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("attempts/").accept(MediaType.APPLICATION_JSON)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         verifyHsTaskAttempts(json, task);
       }
@@ -176,7 +179,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("history")
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("attempts").get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         verifyHsTaskAttempts(json, task);
       }
@@ -197,7 +201,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
             .path("attempts").accept(MediaType.APPLICATION_XML)
             .get(ClientResponse.class);
 
-        assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         String xml = response.getEntity(String.class);
         DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
         DocumentBuilder db = dbf.newDocumentBuilder();
@@ -232,7 +237,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
               .path("mapreduce").path("jobs").path(jobId).path("tasks")
               .path(tid).path("attempts").path(attid)
               .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("taskAttempt");
@@ -261,7 +267,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
               .path("mapreduce").path("jobs").path(jobId).path("tasks")
               .path(tid).path("attempts").path(attid + "/")
               .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("taskAttempt");
@@ -289,7 +296,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
           ClientResponse response = r.path("ws").path("v1").path("history")
               .path("mapreduce").path("jobs").path(jobId).path("tasks")
               .path(tid).path("attempts").path(attid).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("taskAttempt");
@@ -317,7 +325,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
               .path(tid).path("attempts").path(attid)
               .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-          assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+              response.getType().toString());
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
           DocumentBuilder db = dbf.newDocumentBuilder();
@@ -394,7 +403,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
         } catch (UniformInterfaceException ue) {
           ClientResponse response = ue.getResponse();
           assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject msg = response.getEntity(JSONObject.class);
           JSONObject exception = msg.getJSONObject("RemoteException");
           assertEquals("incorrect number of elements", 3, exception.length());
@@ -573,7 +583,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
               .path("mapreduce").path("jobs").path(jobId).path("tasks")
               .path(tid).path("attempts").path(attid).path("counters")
               .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           JSONObject info = json.getJSONObject("jobTaskAttemptCounters");
@@ -601,7 +612,8 @@ public class TestHsWebServicesAttempts extends JerseyTestBase {
               .path(tid).path("attempts").path(attid).path("counters")
               .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-          assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+              response.getType().toString());
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
           DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 53c5adc..86203ce 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -167,7 +168,8 @@ public class TestHsWebServicesJobConf extends JerseyTestBase {
           .path("mapreduce")
           .path("jobs").path(jobId).path("conf")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("conf");
@@ -185,7 +187,8 @@ public class TestHsWebServicesJobConf extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history").path("mapreduce")
           .path("jobs").path(jobId).path("conf/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("conf");
@@ -202,7 +205,8 @@ public class TestHsWebServicesJobConf extends JerseyTestBase {
 
       ClientResponse response = r.path("ws").path("v1").path("history").path("mapreduce")
           .path("jobs").path(jobId).path("conf").get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("conf");
@@ -220,7 +224,8 @@ public class TestHsWebServicesJobConf extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history").path("mapreduce")
           .path("jobs").path(jobId).path("conf")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 665815b..9fe87fd 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
@@ -36,6 +36,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -127,7 +128,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -145,7 +147,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -162,7 +165,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -180,7 +184,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -271,7 +276,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId)
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("job");
@@ -290,7 +296,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId + "/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("job");
@@ -308,7 +315,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
 
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("job");
@@ -328,7 +336,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -356,7 +365,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -380,7 +390,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -404,7 +415,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String msg = response.getEntity(String.class);
       System.out.println(msg);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -443,7 +455,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -470,7 +483,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId)
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -493,7 +507,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("counters")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -511,7 +526,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("counters/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -549,7 +565,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("counters/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -569,7 +586,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("counters/")
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobCounters");
@@ -587,7 +605,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("counters")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -686,7 +705,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobAttempts");
@@ -704,7 +724,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobAttempts");
@@ -722,7 +743,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts")
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject info = json.getJSONObject("jobAttempts");
@@ -740,7 +762,8 @@ public class TestHsWebServicesJobs extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("jobattempts")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();


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


[16/50] [abbrv] hadoop git commit: YARN-4710. Reduce logging application reserved debug info in FSAppAttempt#assignContainer (Contributed by Yiqun Lin via Daniel Templeton)

Posted by vi...@apache.org.
YARN-4710. Reduce logging application reserved debug info in FSAppAttempt#assignContainer (Contributed by Yiqun Lin via Daniel Templeton)


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

Branch: refs/heads/HDFS-9806
Commit: b98fc8249f0576e7b4e230ffc3cec5a20eefc543
Parents: 9449519
Author: Daniel Templeton <te...@apache.org>
Authored: Thu Oct 27 14:35:38 2016 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Thu Oct 27 14:42:19 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b98fc824/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 3555faa..cef4387 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -771,8 +771,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
 
   private Resource assignContainer(FSSchedulerNode node, boolean reserved) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Node offered to app: " + getName() + " reserved: " + reserved);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Node offered to app: " + getName() + " reserved: " + reserved);
     }
 
     Collection<SchedulerRequestKey> keysToTry = (reserved) ?


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


[04/50] [abbrv] hadoop git commit: Revert "HADOOP-13514. Upgrade maven surefire plugin to 2.19.1. Contributed by Ewan Higgs."

Posted by vi...@apache.org.
Revert "HADOOP-13514. Upgrade maven surefire plugin to 2.19.1. Contributed by Ewan Higgs."

This reverts commit dbd205762ef2cba903b9bd9335bb9a5964d51f74.


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

Branch: refs/heads/HDFS-9806
Commit: b43951750254290b0aaec3641cff3061a3927991
Parents: 94e77e9
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 27 15:18:33 2016 +0200
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Oct 27 15:18:33 2016 +0200

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4395175/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index c7c5a72..f914f92 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -107,7 +107,7 @@
 
     <!-- Plugin versions and config -->
     <maven-surefire-plugin.argLine>-Xmx2048m -XX:+HeapDumpOnOutOfMemoryError</maven-surefire-plugin.argLine>
-    <maven-surefire-plugin.version>2.19.1</maven-surefire-plugin.version>
+    <maven-surefire-plugin.version>2.17</maven-surefire-plugin.version>
     <maven-surefire-report-plugin.version>${maven-surefire-plugin.version}</maven-surefire-report-plugin.version>
     <maven-failsafe-plugin.version>${maven-surefire-plugin.version}</maven-failsafe-plugin.version>
 


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


[25/50] [abbrv] hadoop git commit: HDFS-11047. Remove deep copies of FinalizedReplica to alleviate heap consumption on DataNode. Contributed by Xiaobing Zhou

Posted by vi...@apache.org.
HDFS-11047. Remove deep copies of FinalizedReplica to alleviate heap consumption on DataNode. Contributed by Xiaobing Zhou


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

Branch: refs/heads/HDFS-9806
Commit: 9e03ee527988ff85af7f2c224c5570b69d09279a
Parents: f3ac1f4
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Oct 27 15:58:09 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Oct 27 16:00:27 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/DirectoryScanner.java      | 14 +++++++-------
 .../server/datanode/fsdataset/FsDatasetSpi.java     | 11 ++++++++++-
 .../datanode/fsdataset/impl/FsDatasetImpl.java      | 16 +++++++++++-----
 3 files changed, 28 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e03ee52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 58071dc..e2baf32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -398,14 +399,13 @@ public class DirectoryScanner implements Runnable {
         diffs.put(bpid, diffRecord);
         
         statsRecord.totalBlocks = blockpoolReport.length;
-        List<ReplicaInfo> bl = dataset.getFinalizedBlocks(bpid);
-        ReplicaInfo[] memReport = bl.toArray(new ReplicaInfo[bl.size()]);
-        Arrays.sort(memReport); // Sort based on blockId
+        final List<ReplicaInfo> bl = dataset.getFinalizedBlocks(bpid);
+        Collections.sort(bl); // Sort based on blockId
   
         int d = 0; // index for blockpoolReport
         int m = 0; // index for memReprot
-        while (m < memReport.length && d < blockpoolReport.length) {
-          ReplicaInfo memBlock = memReport[m];
+        while (m < bl.size() && d < blockpoolReport.length) {
+          ReplicaInfo memBlock = bl.get(m);
           ScanInfo info = blockpoolReport[d];
           if (info.getBlockId() < memBlock.getBlockId()) {
             if (!dataset.isDeletingBlock(bpid, info.getBlockId())) {
@@ -452,8 +452,8 @@ public class DirectoryScanner implements Runnable {
             ++m;
           }
         }
-        while (m < memReport.length) {
-          ReplicaInfo current = memReport[m++];
+        while (m < bl.size()) {
+          ReplicaInfo current = bl.get(m++);
           addDifference(diffRecord, statsRecord,
                         current.getBlockId(), current.getVolume());
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e03ee52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index f2ffa83..e113212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -229,7 +229,16 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    */
   VolumeFailureSummary getVolumeFailureSummary();
 
-  /** @return a list of finalized blocks for the given block pool. */
+  /**
+   * Gets a list of references to the finalized blocks for the given block pool.
+   * <p>
+   * Callers of this function should call
+   * {@link FsDatasetSpi#acquireDatasetLock} to avoid blocks' status being
+   * changed during list iteration.
+   * </p>
+   * @return a list of references to the finalized blocks for the given block
+   *         pool.
+   */
   List<ReplicaInfo> getFinalizedBlocks(String bpid);
 
   /** @return a list of finalized blocks for the given block pool. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e03ee52/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 abeda2c..0f40f2a 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
@@ -1714,17 +1714,23 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   /**
-   * Get the list of finalized blocks from in-memory blockmap for a block pool.
+   * Gets a list of references to the finalized blocks for the given block pool.
+   * <p>
+   * Callers of this function should call
+   * {@link FsDatasetSpi#acquireDatasetLock} to avoid blocks' status being
+   * changed during list iteration.
+   * </p>
+   * @return a list of references to the finalized blocks for the given block
+   *         pool.
    */
   @Override
   public List<ReplicaInfo> getFinalizedBlocks(String bpid) {
     try (AutoCloseableLock lock = datasetLock.acquire()) {
-      ArrayList<ReplicaInfo> finalized =
-          new ArrayList<ReplicaInfo>(volumeMap.size(bpid));
+      final List<ReplicaInfo> finalized = new ArrayList<ReplicaInfo>(
+          volumeMap.size(bpid));
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
         if (b.getState() == ReplicaState.FINALIZED) {
-          finalized.add(new ReplicaBuilder(ReplicaState.FINALIZED)
-              .from(b).build());
+          finalized.add(b);
         }
       }
       return finalized;


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


[33/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js.gz
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js.gz b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js.gz
deleted file mode 100644
index d2e3ec8..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js.gz and /dev/null differ


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


[49/50] [abbrv] hadoop git commit: YARN-5767. Fix the order that resources are cleaned up from the local Public/Private caches. Contributed by Chris Trezzo

Posted by vi...@apache.org.
YARN-5767. Fix the order that resources are cleaned up from the local Public/Private caches. Contributed by Chris Trezzo


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

Branch: refs/heads/HDFS-9806
Commit: 1b79c417dca17bcd2e031864bc6ca07254c61b47
Parents: 7146359
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Oct 28 15:58:04 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Oct 28 15:58:04 2016 +0000

----------------------------------------------------------------------
 .../localizer/LocalCacheCleaner.java            | 182 ++++++++++++++
 .../localizer/LocalResourcesTrackerImpl.java    |   3 +-
 .../localizer/ResourceLocalizationService.java  |  37 +--
 .../localizer/ResourceRetentionSet.java         |  96 --------
 .../localizer/TestLocalCacheCleanup.java        | 235 +++++++++++++++++++
 .../localizer/TestResourceRetention.java        | 106 ---------
 6 files changed, 440 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b79c417/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheCleaner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheCleaner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheCleaner.java
new file mode 100644
index 0000000..7311919
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheCleaner.java
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+
+/**
+ * A class responsible for cleaning the PUBLIC and PRIVATE local caches on a
+ * node manager.
+ */
+class LocalCacheCleaner {
+
+  private long currentSize;
+  private final long targetSize;
+  private final DeletionService delService;
+  private final SortedMap<LocalizedResource, LocalResourcesTracker> resourceMap;
+
+  LocalCacheCleaner(DeletionService delService, long targetSize) {
+    this(delService, targetSize, new LRUComparator());
+  }
+
+  LocalCacheCleaner(DeletionService delService, long targetSize,
+      Comparator<? super LocalizedResource> cmp) {
+    this(delService, targetSize,
+        new TreeMap<LocalizedResource, LocalResourcesTracker>(cmp));
+  }
+
+  LocalCacheCleaner(DeletionService delService, long targetSize,
+      SortedMap<LocalizedResource, LocalResourcesTracker> resourceMap) {
+    this.resourceMap = resourceMap;
+    this.delService = delService;
+    this.targetSize = targetSize;
+  }
+
+  /**
+   * Adds resources from the passed LocalResourceTracker that are candidates for
+   * deletion from the cache.
+   *
+   * @param newTracker add all resources being tracked by the passed
+   *          LocalResourcesTracker to the LocalCacheCleaner.
+   */
+  public void addResources(LocalResourcesTracker newTracker) {
+    for (LocalizedResource resource : newTracker) {
+      currentSize += resource.getSize();
+      if (resource.getRefCount() > 0) {
+        // Do not delete resources that are still in use
+        continue;
+      }
+      resourceMap.put(resource, newTracker);
+    }
+  }
+
+  /**
+   * Delete resources from the cache in the sorted order generated by the
+   * Comparator used to construct this class.
+   *
+   * @return stats about what was cleaned up during this call of cleanCache
+   */
+  public LocalCacheCleanerStats cleanCache() {
+    LocalCacheCleanerStats stats = new LocalCacheCleanerStats(currentSize);
+    for (Iterator<Map.Entry<LocalizedResource, LocalResourcesTracker>> i =
+        resourceMap.entrySet().iterator();
+        currentSize - stats.totalDelSize > targetSize && i.hasNext();) {
+      Map.Entry<LocalizedResource, LocalResourcesTracker> rsrc = i.next();
+      LocalizedResource resource = rsrc.getKey();
+      LocalResourcesTracker tracker = rsrc.getValue();
+      if (tracker.remove(resource, delService)) {
+        stats.incDelSize(tracker.getUser(), resource.getSize());
+      }
+    }
+    this.resourceMap.clear();
+    return stats;
+  }
+
+  static class LocalCacheCleanerStats {
+    private final Map<String, Long> userDelSizes = new TreeMap<String, Long>();
+    private final long cacheSizeBeforeClean;
+    private long totalDelSize;
+    private long publicDelSize;
+    private long privateDelSize;
+
+    LocalCacheCleanerStats(long cacheSizeBeforeClean) {
+      this.cacheSizeBeforeClean = cacheSizeBeforeClean;
+    }
+
+    void incDelSize(String user, long delSize) {
+      totalDelSize += delSize;
+      if (user == null) {
+        publicDelSize += delSize;
+      } else {
+        privateDelSize += delSize;
+        Long userDel = userDelSizes.get(user);
+        if (userDel != null) {
+          userDel += delSize;
+          userDelSizes.put(user, userDel);
+        } else {
+          userDelSizes.put(user, delSize);
+        }
+      }
+    }
+
+    Map<String, Long> getUserDelSizes() {
+      return Collections.unmodifiableMap(userDelSizes);
+    }
+
+    long getCacheSizeBeforeClean() {
+      return cacheSizeBeforeClean;
+    }
+
+    long getTotalDelSize() {
+      return totalDelSize;
+    }
+
+    long getPublicDelSize() {
+      return publicDelSize;
+    }
+
+    long getPrivateDelSize() {
+      return privateDelSize;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("Cache Size Before Clean: ").append(cacheSizeBeforeClean)
+          .append(", ");
+      sb.append("Total Deleted: ").append(totalDelSize).append(", ");
+      sb.append("Public Deleted: ").append(publicDelSize).append(", ");
+      sb.append("Private Deleted: ").append(privateDelSize);
+      return sb.toString();
+    }
+
+    public String toStringDetailed() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(this.toString());
+      sb.append(", Private Deleted Detail: {");
+      for (Map.Entry<String, Long> e : userDelSizes.entrySet()) {
+        sb.append(" ").append(e.getKey()).append(":").append(e.getValue());
+      }
+      sb.append(" }");
+      return sb.toString();
+    }
+  }
+
+  private static class LRUComparator implements Comparator<LocalizedResource>,
+      Serializable {
+
+    private static final long serialVersionUID = 7034380228434701685L;
+
+    public int compare(LocalizedResource r1, LocalizedResource r2) {
+      long ret = r1.getTimestamp() - r2.getTimestamp();
+      if (0 == ret) {
+        return System.identityHashCode(r1) - System.identityHashCode(r2);
+      }
+      return ret > 0 ? 1 : -1;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b79c417/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
index 38fffe6..940c599 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
@@ -65,7 +65,8 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   private final String user;
   private final ApplicationId appId;
   private final Dispatcher dispatcher;
-  private final ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc;
+  @VisibleForTesting
+  final ConcurrentMap<LocalResourceRequest, LocalizedResource> localrsrc;
   private Configuration conf;
   private LocalDirsHandlerService dirsHandler;
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b79c417/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 4bd004b..4cd1acc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -113,6 +113,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerResourceFailedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalCacheCleaner.LocalCacheCleanerStats;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
@@ -152,7 +153,8 @@ public class ResourceLocalizationService extends CompositeService
 
   private Server server;
   private InetSocketAddress localizationServerAddress;
-  private long cacheTargetSize;
+  @VisibleForTesting
+  long cacheTargetSize;
   private long cacheCleanupPeriod;
 
   private final ContainerExecutor exec;
@@ -164,7 +166,8 @@ public class ResourceLocalizationService extends CompositeService
   private LocalizerTokenSecretManager secretManager;
   private NMStateStoreService stateStore;
 
-  private LocalResourcesTracker publicRsrc;
+  @VisibleForTesting
+  LocalResourcesTracker publicRsrc;
 
   private LocalDirsHandlerService dirsHandler;
   private DirsChangeListener localDirsChangeListener;
@@ -176,7 +179,8 @@ public class ResourceLocalizationService extends CompositeService
    * Map of LocalResourceTrackers keyed by username, for private
    * resources.
    */
-  private final ConcurrentMap<String,LocalResourcesTracker> privateRsrc =
+  @VisibleForTesting
+  final ConcurrentMap<String, LocalResourcesTracker> privateRsrc =
     new ConcurrentHashMap<String,LocalResourcesTracker>();
 
   /**
@@ -427,7 +431,7 @@ public class ResourceLocalizationService extends CompositeService
       handleContainerResourcesLocalized((ContainerLocalizationEvent) event);
       break;
     case CACHE_CLEANUP:
-      handleCacheCleanup(event);
+      handleCacheCleanup();
       break;
     case CLEANUP_CONTAINER_RESOURCES:
       handleCleanupContainerResources((ContainerLocalizationCleanupEvent)event);
@@ -512,20 +516,21 @@ public class ResourceLocalizationService extends CompositeService
     localizerTracker.endContainerLocalization(locId);
   }
 
-  private void handleCacheCleanup(LocalizationEvent event) {
-    ResourceRetentionSet retain =
-      new ResourceRetentionSet(delService, cacheTargetSize);
-    retain.addResources(publicRsrc);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Resource cleanup (public) " + retain);
-    }
+  @VisibleForTesting
+  LocalCacheCleanerStats handleCacheCleanup() {
+    LocalCacheCleaner cleaner =
+        new LocalCacheCleaner(delService, cacheTargetSize);
+    cleaner.addResources(publicRsrc);
     for (LocalResourcesTracker t : privateRsrc.values()) {
-      retain.addResources(t);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Resource cleanup " + t.getUser() + ":" + retain);
-      }
+      cleaner.addResources(t);
+    }
+    LocalCacheCleaner.LocalCacheCleanerStats stats = cleaner.cleanCache();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(stats.toStringDetailed());
+    } else if (LOG.isInfoEnabled()) {
+      LOG.info(stats.toString());
     }
-    //TODO Check if appRsrcs should also be added to the retention set.
+    return stats;
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b79c417/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceRetentionSet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceRetentionSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceRetentionSet.java
deleted file mode 100644
index 447a792..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceRetentionSet.java
+++ /dev/null
@@ -1,96 +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.yarn.server.nodemanager.containermanager.localizer;
-
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
-
-public class ResourceRetentionSet {
-
-  private long delSize;
-  private long currentSize;
-  private final long targetSize;
-  private final DeletionService delService;
-  private final SortedMap<LocalizedResource,LocalResourcesTracker> retain;
-
-  ResourceRetentionSet(DeletionService delService, long targetSize) {
-    this(delService, targetSize, new LRUComparator());
-  }
-
-  ResourceRetentionSet(DeletionService delService, long targetSize,
-      Comparator<? super LocalizedResource> cmp) {
-    this(delService, targetSize,
-        new TreeMap<LocalizedResource,LocalResourcesTracker>(cmp));
-  }
-
-  ResourceRetentionSet(DeletionService delService, long targetSize,
-      SortedMap<LocalizedResource,LocalResourcesTracker> retain) {
-    this.retain = retain;
-    this.delService = delService;
-    this.targetSize = targetSize;
-  }
-
-  public void addResources(LocalResourcesTracker newTracker) {
-    for (LocalizedResource resource : newTracker) {
-      currentSize += resource.getSize();
-      if (resource.getRefCount() > 0) {
-        // always retain resources in use
-        continue;
-      }
-      retain.put(resource, newTracker);
-    }
-    for (Iterator<Map.Entry<LocalizedResource,LocalResourcesTracker>> i =
-           retain.entrySet().iterator();
-         currentSize - delSize > targetSize && i.hasNext();) {
-      Map.Entry<LocalizedResource,LocalResourcesTracker> rsrc = i.next();
-      LocalizedResource resource = rsrc.getKey();
-      LocalResourcesTracker tracker = rsrc.getValue();
-      if (tracker.remove(resource, delService)) {
-        delSize += resource.getSize();
-        i.remove();
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("Cache: ").append(currentSize).append(", ");
-    sb.append("Deleted: ").append(delSize);
-    return sb.toString();
-  }
-
-  static class LRUComparator implements Comparator<LocalizedResource> {
-    public int compare(LocalizedResource r1, LocalizedResource r2) {
-      long ret = r1.getTimestamp() - r2.getTimestamp();
-      if (0 == ret) {
-        return System.identityHashCode(r1) - System.identityHashCode(r2);
-      }
-      return ret > 0 ? 1 : -1;
-    }
-    public boolean equals(Object other) {
-      return this == other;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b79c417/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheCleanup.java
new file mode 100644
index 0000000..d6db67a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheCleanup.java
@@ -0,0 +1,235 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalCacheCleaner.LocalCacheCleanerStats;
+import org.junit.Test;
+
+/**
+ * This class tests the clean up of local caches the node manager uses for the
+ * purpose of resource localization.
+ */
+public class TestLocalCacheCleanup {
+
+  @Test
+  public void testBasicCleanup() {
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> publicRsrc =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    addResource(publicRsrc, "/pub-resource1.txt", 5, 20, 0);
+    addResource(publicRsrc, "/pub-resource2.txt", 3, 20, 0);
+    addResource(publicRsrc, "/pub-resource3.txt", 15, 20, 0);
+
+    ConcurrentMap<String, LocalResourcesTracker> privateRsrc =
+        new ConcurrentHashMap<String, LocalResourcesTracker>();
+
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> user1rsrcs =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    addResource(user1rsrcs, "/private-u1-resource4.txt", 1, 20, 0);
+    LocalResourcesTracker user1Tracker =
+        new StubbedLocalResourcesTrackerImpl("user1", user1rsrcs);
+    privateRsrc.put("user1", user1Tracker);
+
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> user2rsrcs =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    addResource(user2rsrcs, "/private-u2-resource5.txt", 2, 20, 0);
+    LocalResourcesTracker user2Tracker =
+        new StubbedLocalResourcesTrackerImpl("user2", user2rsrcs);
+    privateRsrc.put("user2", user2Tracker);
+
+    ResourceLocalizationService rls =
+        createLocService(publicRsrc, privateRsrc, 0);
+    LocalCacheCleanerStats stats = rls.handleCacheCleanup();
+    assertEquals(0, ((StubbedLocalResourcesTrackerImpl) rls.publicRsrc)
+        .getLocalRsrc().size());
+    assertEquals(0,
+        ((StubbedLocalResourcesTrackerImpl) privateRsrc.get("user1"))
+            .getLocalRsrc().size());
+    assertEquals(0,
+        ((StubbedLocalResourcesTrackerImpl) privateRsrc.get("user2"))
+            .getLocalRsrc().size());
+    assertEquals(100, stats.getTotalDelSize());
+    assertEquals(60, stats.getPublicDelSize());
+    assertEquals(40, stats.getPrivateDelSize());
+  }
+
+  @Test
+  public void testPositiveRefCount() {
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> publicRsrc =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    // Oldest resource with a positive ref count the other with a ref count
+    // equal to 0.
+    LocalResourceRequest survivor =
+        addResource(publicRsrc, "/pub-resource1.txt", 1, 20, 1);
+    addResource(publicRsrc, "/pub-resource2.txt", 5, 20, 0);
+
+    ConcurrentMap<String, LocalResourcesTracker> privateRsrc =
+        new ConcurrentHashMap<String, LocalResourcesTracker>();
+
+    ResourceLocalizationService rls =
+        createLocService(publicRsrc, privateRsrc, 0);
+    LocalCacheCleanerStats stats = rls.handleCacheCleanup();
+    StubbedLocalResourcesTrackerImpl resources =
+        (StubbedLocalResourcesTrackerImpl) rls.publicRsrc;
+    assertEquals(1, resources.getLocalRsrc().size());
+    assertTrue(resources.getLocalRsrc().containsKey(survivor));
+    assertEquals(20, stats.getTotalDelSize());
+    assertEquals(20, stats.getPublicDelSize());
+    assertEquals(0, stats.getPrivateDelSize());
+  }
+
+  @Test
+  public void testLRUAcrossTrackers() {
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> publicRsrc =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    LocalResourceRequest pubSurviver1 =
+        addResource(publicRsrc, "/pub-resource1.txt", 8, 20, 0);
+    LocalResourceRequest pubSurviver2 =
+        addResource(publicRsrc, "/pub-resource2.txt", 7, 20, 0);
+    addResource(publicRsrc, "/pub-resource3.txt", 1, 20, 0);
+
+    ConcurrentMap<String, LocalResourcesTracker> privateRsrc =
+        new ConcurrentHashMap<String, LocalResourcesTracker>();
+
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> user1rsrcs =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    LocalResourceRequest usr1Surviver1 =
+        addResource(user1rsrcs, "/private-u1-resource1.txt", 6, 20, 0);
+    addResource(user1rsrcs, "/private-u1-resource2.txt", 2, 20, 0);
+    LocalResourcesTracker user1Tracker =
+        new StubbedLocalResourcesTrackerImpl("user1", user1rsrcs);
+    privateRsrc.put("user1", user1Tracker);
+
+    ConcurrentMap<LocalResourceRequest, LocalizedResource> user2rsrcs =
+        new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+    LocalResourceRequest usr2Surviver1 =
+        addResource(user2rsrcs, "/private-u2-resource1.txt", 5, 20, 0);
+    addResource(user2rsrcs, "/private-u2-resource2.txt", 3, 20, 0);
+    addResource(user2rsrcs, "/private-u2-resource3.txt", 4, 20, 0);
+    LocalResourcesTracker user2Tracker =
+        new StubbedLocalResourcesTrackerImpl("user2", user2rsrcs);
+    privateRsrc.put("user2", user2Tracker);
+
+    ResourceLocalizationService rls =
+        createLocService(publicRsrc, privateRsrc, 80);
+    LocalCacheCleanerStats stats = rls.handleCacheCleanup();
+
+    Map<LocalResourceRequest, LocalizedResource> pubLocalRsrc =
+        ((StubbedLocalResourcesTrackerImpl) rls.publicRsrc).getLocalRsrc();
+    assertEquals(2, pubLocalRsrc.size());
+    assertTrue(pubLocalRsrc.containsKey(pubSurviver1));
+    assertTrue(pubLocalRsrc.containsKey(pubSurviver2));
+
+    Map<LocalResourceRequest, LocalizedResource> usr1LocalRsrc =
+        ((StubbedLocalResourcesTrackerImpl) privateRsrc.get("user1"))
+            .getLocalRsrc();
+    assertEquals(1, usr1LocalRsrc.size());
+    assertTrue(usr1LocalRsrc.containsKey(usr1Surviver1));
+
+    Map<LocalResourceRequest, LocalizedResource> usr2LocalRsrc =
+        ((StubbedLocalResourcesTrackerImpl) privateRsrc.get("user2"))
+            .getLocalRsrc();
+    assertEquals(1, usr2LocalRsrc.size());
+    assertTrue(usr2LocalRsrc.containsKey(usr2Surviver1));
+
+    assertEquals(80, stats.getTotalDelSize());
+    assertEquals(20, stats.getPublicDelSize());
+    assertEquals(60, stats.getPrivateDelSize());
+  }
+
+  private ResourceLocalizationService createLocService(
+      ConcurrentMap<LocalResourceRequest, LocalizedResource> publicRsrcs,
+      ConcurrentMap<String, LocalResourcesTracker> privateRsrcs,
+      long targetCacheSize) {
+    Context mockedContext = mock(Context.class);
+    when(mockedContext.getNMStateStore()).thenReturn(null);
+    ResourceLocalizationService rls =
+        new ResourceLocalizationService(null, null, null, null, mockedContext);
+    // We set the following members directly so we don't have to deal with
+    // mocking out the service init method.
+    rls.publicRsrc = new StubbedLocalResourcesTrackerImpl(null, publicRsrcs);
+    rls.cacheTargetSize = targetCacheSize;
+    rls.privateRsrc.putAll(privateRsrcs);
+    return rls;
+  }
+
+  private LocalResourceRequest addResource(
+      ConcurrentMap<LocalResourceRequest, LocalizedResource> resources,
+      String path, long timestamp, long size, int refCount) {
+    LocalResourceRequest request = createLocalResourceRequest(path, timestamp);
+    LocalizedResource resource =
+        createLocalizedResource(size, refCount, timestamp, request);
+    resources.put(request, resource);
+    return request;
+  }
+
+  private LocalResourceRequest createLocalResourceRequest(String path,
+      long timestamp) {
+    return new LocalResourceRequest(new Path(path), timestamp,
+        LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, null);
+  }
+
+  private LocalizedResource createLocalizedResource(long size, int refCount,
+      long timestamp, LocalResourceRequest req) {
+    LocalizedResource lr = mock(LocalizedResource.class);
+    when(lr.getSize()).thenReturn(size);
+    when(lr.getRefCount()).thenReturn(refCount);
+    when(lr.getTimestamp()).thenReturn(timestamp);
+    when(lr.getState()).thenReturn(ResourceState.LOCALIZED);
+    when(lr.getRequest()).thenReturn(req);
+    return lr;
+  }
+
+  class StubbedLocalResourcesTrackerImpl extends LocalResourcesTrackerImpl {
+    StubbedLocalResourcesTrackerImpl(String user,
+        ConcurrentMap<LocalResourceRequest, LocalizedResource> rsrcs) {
+      super(user, null, null, rsrcs, false, new Configuration(), null, null);
+    }
+
+    @Override
+    public boolean remove(LocalizedResource rem, DeletionService delService) {
+      LocalizedResource r = localrsrc.remove(rem.getRequest());
+      if (r != null) {
+        LOG.info("Removed " + rem.getRequest().getPath()
+            + " from localized cache");
+        return true;
+      }
+      return false;
+    }
+
+    Map<LocalResourceRequest, LocalizedResource> getLocalRsrc() {
+      return Collections.unmodifiableMap(localrsrc);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b79c417/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceRetention.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceRetention.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceRetention.java
deleted file mode 100644
index 81e69e2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceRetention.java
+++ /dev/null
@@ -1,106 +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.yarn.server.nodemanager.containermanager.localizer;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
-import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-import org.mockito.ArgumentCaptor;
-
-import static org.mockito.Mockito.*;
-
-public class TestResourceRetention {
-
-  @Test
-  public void testRsrcUnused() {
-    DeletionService delService = mock(DeletionService.class);
-    long TARGET_MB = 10 << 20;
-    ResourceRetentionSet rss = new ResourceRetentionSet(delService, TARGET_MB);
-    // 3MB files @{10, 15}
-    LocalResourcesTracker pubTracker =
-      createMockTracker(null, 3 * 1024 * 1024, 2, 10, 5);
-    // 1MB files @{3, 6, 9, 12}
-    LocalResourcesTracker trackerA =
-      createMockTracker("A", 1 * 1024 * 1024, 4, 3, 3);
-    // 4MB file @{1}
-    LocalResourcesTracker trackerB =
-      createMockTracker("B", 4 * 1024 * 1024, 1, 10, 5);
-    // 2MB files @{7, 9, 11}
-    LocalResourcesTracker trackerC =
-      createMockTracker("C", 2 * 1024 * 1024, 3, 7, 2);
-    // Total cache: 20MB; verify removed at least 10MB
-    rss.addResources(pubTracker);
-    rss.addResources(trackerA);
-    rss.addResources(trackerB);
-    rss.addResources(trackerC);
-    long deleted = 0L;
-    ArgumentCaptor<LocalizedResource> captor =
-      ArgumentCaptor.forClass(LocalizedResource.class);
-    verify(pubTracker, atMost(2))
-      .remove(captor.capture(), isA(DeletionService.class));
-    verify(trackerA, atMost(4))
-      .remove(captor.capture(), isA(DeletionService.class));
-    verify(trackerB, atMost(1))
-      .remove(captor.capture(), isA(DeletionService.class));
-    verify(trackerC, atMost(3))
-      .remove(captor.capture(), isA(DeletionService.class));
-    for (LocalizedResource rem : captor.getAllValues()) {
-      deleted += rem.getSize();
-    }
-    assertTrue(deleted >= 10 * 1024 * 1024);
-    assertTrue(deleted < 15 * 1024 * 1024);
-  }
-
-  LocalResourcesTracker createMockTracker(String user, final long rsrcSize,
-      long nRsrcs, long timestamp, long tsstep) {
-    Configuration conf = new Configuration();
-    ConcurrentMap<LocalResourceRequest,LocalizedResource> trackerResources =
-      new ConcurrentHashMap<LocalResourceRequest,LocalizedResource>();
-    LocalResourcesTracker ret = spy(new LocalResourcesTrackerImpl(user, null,
-      null, trackerResources, false, conf, new NMNullStateStoreService(),null));
-    for (int i = 0; i < nRsrcs; ++i) {
-      final LocalResourceRequest req = new LocalResourceRequest(
-          new Path("file:///" + user + "/rsrc" + i), timestamp + i * tsstep,
-          LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, null);
-      final long ts = timestamp + i * tsstep;
-      final Path p = new Path("file:///local/" + user + "/rsrc" + i);
-      LocalizedResource rsrc = new LocalizedResource(req, null) {
-        @Override public int getRefCount() { return 0; }
-        @Override public long getSize() { return rsrcSize; }
-        @Override public Path getLocalPath() { return p; }
-        @Override public long getTimestamp() { return ts; }
-        @Override
-        public ResourceState getState() { return ResourceState.LOCALIZED; }
-      };
-      trackerResources.put(req, rsrc);
-    }
-    return ret;
-  }
-
-}


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


[40/50] [abbrv] hadoop git commit: MAPREDUCE-2631. Potential resource leaks in BinaryProtocol$TeeOutputStream.java. Contributed by Sunil G.

Posted by vi...@apache.org.
MAPREDUCE-2631. Potential resource leaks in BinaryProtocol$TeeOutputStream.java. Contributed by Sunil G.


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

Branch: refs/heads/HDFS-9806
Commit: 28660f51af161a9fa301523d96a6f8ae4ebd6edd
Parents: 5877f20
Author: Naganarasimha <na...@apache.org>
Authored: Fri Oct 28 05:50:13 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Fri Oct 28 05:50:13 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/mapred/IFileOutputStream.java |  8 ++++++--
 .../hadoop/mapred/pipes/BinaryProtocol.java     | 14 +++++++++-----
 .../apache/hadoop/mapred/TestIFileStreams.java  | 20 ++++++++++++++++++++
 3 files changed, 35 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28660f51/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFileOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFileOutputStream.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFileOutputStream.java
index 8f25ba7..08bcd24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFileOutputStream.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFileOutputStream.java
@@ -24,6 +24,7 @@ import java.io.FilterOutputStream;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 /**
  * A Checksum output stream.
@@ -60,8 +61,11 @@ public class IFileOutputStream extends FilterOutputStream {
       return;
     }
     closed = true;
-    finish();
-    out.close();
+    try {
+      finish();
+    } finally {
+      IOUtils.closeStream(out);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28660f51/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java
index ebfb184..5a3ed5b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java
@@ -36,6 +36,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -200,8 +201,8 @@ class BinaryProtocol<K1 extends WritableComparable, V1 extends Writable,
       file = new FileOutputStream(filename);
     }
     public void write(byte b[], int off, int len) throws IOException {
-      file.write(b,off,len);
-      out.write(b,off,len);
+      file.write(b, off, len);
+      out.write(b, off, len);
     }
 
     public void write(int b) throws IOException {
@@ -215,9 +216,12 @@ class BinaryProtocol<K1 extends WritableComparable, V1 extends Writable,
     }
 
     public void close() throws IOException {
-      flush();
-      file.close();
-      out.close();
+      try {
+        flush();
+      } finally {
+        IOUtils.closeStream(file);
+        IOUtils.closeStream(out);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28660f51/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java
index 2b97d3b..a815b28 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java
@@ -22,7 +22,13 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.junit.Test;
+import org.mockito.Mockito;
+
 import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
 import static org.junit.Assert.assertEquals;
 
 public class TestIFileStreams {
@@ -99,4 +105,18 @@ public class TestIFileStreams {
     fail("Did not detect bad data in checksum");
   }
 
+  @Test
+  public void testCloseStreamOnException() throws Exception {
+    OutputStream outputStream = Mockito.mock(OutputStream.class);
+    IFileOutputStream ifos = new IFileOutputStream(outputStream);
+    Mockito.doThrow(new IOException("Dummy Exception")).when(outputStream)
+        .flush();
+    try {
+      ifos.close();
+      fail("IOException is not thrown");
+    } catch (IOException ioe) {
+      assertEquals("Dummy Exception", ioe.getMessage());
+    }
+    Mockito.verify(outputStream).close();
+  }
 }


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


[23/50] [abbrv] hadoop git commit: YARN-4765 Split TestHBaseTimelineStorage into multiple test classes (Varun Saxena via Vrushali C)

Posted by vi...@apache.org.
YARN-4765 Split TestHBaseTimelineStorage into multiple test classes (Varun Saxena via Vrushali C)

(cherry picked from commit 513dcf6817dd76fde8096ff04cd888d7c908461d)


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

Branch: refs/heads/HDFS-9806
Commit: 022bf783aa89c1c81374ebef5dba2df95b7563b5
Parents: 221582c
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Thu Oct 27 14:37:50 2016 -0700
Committer: Sangjin Lee <sj...@apache.org>
Committed: Thu Oct 27 15:37:36 2016 -0700

----------------------------------------------------------------------
 .../storage/DataGeneratorForTest.java           |  381 ++
 .../storage/TestHBaseTimelineStorage.java       | 3751 ------------------
 .../storage/TestHBaseTimelineStorageApps.java   | 1849 +++++++++
 .../TestHBaseTimelineStorageEntities.java       | 1675 ++++++++
 4 files changed, 3905 insertions(+), 3751 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/022bf783/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java
new file mode 100644
index 0000000..0938e9e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java
@@ -0,0 +1,381 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
+
+final class DataGeneratorForTest {
+  static void loadApps(HBaseTestingUtility util) throws IOException {
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "application_1111111111_2222";
+    entity.setId(id);
+    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    Long cTime = 1425016502000L;
+    entity.setCreatedTime(cTime);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<>();
+    infoMap.put("infoMapKey1", "infoMapValue2");
+    infoMap.put("infoMapKey2", 20);
+    infoMap.put("infoMapKey3", 85.85);
+    entity.addInfo(infoMap);
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet = new HashSet<>();
+    isRelatedToSet.add("relatedto1");
+    Map<String, Set<String>> isRelatedTo = new HashMap<>();
+    isRelatedTo.put("task", isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+    // add the relatesTo info
+    Set<String> relatesToSet = new HashSet<>();
+    relatesToSet.add("relatesto1");
+    relatesToSet.add("relatesto3");
+    Map<String, Set<String>> relatesTo = new HashMap<>();
+    relatesTo.put("container", relatesToSet);
+    Set<String> relatesToSet11 = new HashSet<>();
+    relatesToSet11.add("relatesto4");
+    relatesTo.put("container1", relatesToSet11);
+    entity.setRelatesToEntities(relatesTo);
+    // add some config entries
+    Map<String, String> conf = new HashMap<>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    conf.put("cfg_param1", "value3");
+    entity.addConfigs(conf);
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricValues = new HashMap<>();
+    long ts = System.currentTimeMillis();
+    metricValues.put(ts - 120000, 100000000);
+    metricValues.put(ts - 100000, 200000000);
+    metricValues.put(ts - 80000, 300000000);
+    metricValues.put(ts - 60000, 400000000);
+    metricValues.put(ts - 40000, 50000000000L);
+    metricValues.put(ts - 20000, 60000000000L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m12 = new TimelineMetric();
+    m12.setId("MAP1_BYTES");
+    m12.addValue(ts, 50);
+    metrics.add(m12);
+    entity.addMetrics(metrics);
+    TimelineEvent event = new TimelineEvent();
+    event.setId("start_event");
+    event.setTimestamp(ts);
+    entity.addEvent(event);
+    te.addEntity(entity);
+    TimelineEntities te1 = new TimelineEntities();
+    TimelineEntity entity1 = new TimelineEntity();
+    String id1 = "application_1111111111_3333";
+    entity1.setId(id1);
+    entity1.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    entity1.setCreatedTime(cTime + 20L);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap1 = new HashMap<>();
+    infoMap1.put("infoMapKey1", "infoMapValue1");
+    infoMap1.put("infoMapKey2", 10);
+    entity1.addInfo(infoMap1);
+
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet1 = new HashSet<>();
+    isRelatedToSet1.add("relatedto3");
+    isRelatedToSet1.add("relatedto5");
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<>();
+    isRelatedTo1.put("task1", isRelatedToSet1);
+    Set<String> isRelatedToSet11 = new HashSet<>();
+    isRelatedToSet11.add("relatedto4");
+    isRelatedTo1.put("task2", isRelatedToSet11);
+    entity1.setIsRelatedToEntities(isRelatedTo1);
+
+    // add the relatesTo info
+    Set<String> relatesToSet1 = new HashSet<>();
+    relatesToSet1.add("relatesto1");
+    relatesToSet1.add("relatesto2");
+    Map<String, Set<String>> relatesTo1 = new HashMap<>();
+    relatesTo1.put("container", relatesToSet1);
+    entity1.setRelatesToEntities(relatesTo1);
+
+    // add some config entries
+    Map<String, String> conf1 = new HashMap<>();
+    conf1.put("cfg_param1", "value1");
+    conf1.put("cfg_param2", "value2");
+    entity1.addConfigs(conf1);
+
+    // add metrics
+    Set<TimelineMetric> metrics1 = new HashSet<>();
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId("MAP1_SLOT_MILLIS");
+    Map<Long, Number> metricValues1 = new HashMap<>();
+    long ts1 = System.currentTimeMillis();
+    metricValues1.put(ts1 - 120000, 100000000);
+    metricValues1.put(ts1 - 100000, 200000000);
+    metricValues1.put(ts1 - 80000, 300000000);
+    metricValues1.put(ts1 - 60000, 400000000);
+    metricValues1.put(ts1 - 40000, 50000000000L);
+    metricValues1.put(ts1 - 20000, 60000000000L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues1);
+    metrics1.add(m2);
+    entity1.addMetrics(metrics1);
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId("end_event");
+    event11.setTimestamp(ts);
+    entity1.addEvent(event11);
+    TimelineEvent event12 = new TimelineEvent();
+    event12.setId("update_event");
+    event12.setTimestamp(ts - 10);
+    entity1.addEvent(event12);
+    te1.addEntity(entity1);
+
+    TimelineEntities te2 = new TimelineEntities();
+    TimelineEntity entity2 = new TimelineEntity();
+    String id2 = "application_1111111111_4444";
+    entity2.setId(id2);
+    entity2.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    entity2.setCreatedTime(cTime + 40L);
+    TimelineEvent event21 = new TimelineEvent();
+    event21.setId("update_event");
+    event21.setTimestamp(ts - 20);
+    entity2.addEvent(event21);
+    Set<String> isRelatedToSet2 = new HashSet<String>();
+    isRelatedToSet2.add("relatedto3");
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<>();
+    isRelatedTo2.put("task1", isRelatedToSet2);
+    entity2.setIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo3 = new HashMap<>();
+    Set<String> relatesToSet14 = new HashSet<String>();
+    relatesToSet14.add("relatesto7");
+    relatesTo3.put("container2", relatesToSet14);
+    entity2.setRelatesToEntities(relatesTo3);
+
+    te2.addEntity(entity2);
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
+      hbi.init(util.getConfiguration());
+      hbi.start();
+      String cluster = "cluster1";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      String appName = "application_1111111111_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      appName = "application_1111111111_3333";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+      appName = "application_1111111111_4444";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te2);
+      hbi.stop();
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  static void loadEntities(HBaseTestingUtility util) throws IOException {
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "hello";
+    String type = "world";
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016502000L;
+    entity.setCreatedTime(cTime);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<>();
+    infoMap.put("infoMapKey1", "infoMapValue2");
+    infoMap.put("infoMapKey2", 20);
+    infoMap.put("infoMapKey3", 71.4);
+    entity.addInfo(infoMap);
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet = new HashSet<>();
+    isRelatedToSet.add("relatedto1");
+    Map<String, Set<String>> isRelatedTo = new HashMap<>();
+    isRelatedTo.put("task", isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add("relatesto1");
+    relatesToSet.add("relatesto3");
+    Map<String, Set<String>> relatesTo = new HashMap<>();
+    relatesTo.put("container", relatesToSet);
+    Set<String> relatesToSet11 = new HashSet<>();
+    relatesToSet11.add("relatesto4");
+    relatesTo.put("container1", relatesToSet11);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    conf.put("cfg_param1", "value3");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricValues = new HashMap<>();
+    long ts = System.currentTimeMillis();
+    metricValues.put(ts - 120000, 100000000);
+    metricValues.put(ts - 100000, 200000000);
+    metricValues.put(ts - 80000, 300000000);
+    metricValues.put(ts - 60000, 400000000);
+    metricValues.put(ts - 40000, 50000000000L);
+    metricValues.put(ts - 20000, 70000000000L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m12 = new TimelineMetric();
+    m12.setId("MAP1_BYTES");
+    m12.addValue(ts, 50);
+    metrics.add(m12);
+    entity.addMetrics(metrics);
+    TimelineEvent event = new TimelineEvent();
+    event.setId("start_event");
+    event.setTimestamp(ts);
+    entity.addEvent(event);
+    te.addEntity(entity);
+
+    TimelineEntity entity1 = new TimelineEntity();
+    String id1 = "hello1";
+    entity1.setId(id1);
+    entity1.setType(type);
+    entity1.setCreatedTime(cTime + 20L);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap1 = new HashMap<>();
+    infoMap1.put("infoMapKey1", "infoMapValue1");
+    infoMap1.put("infoMapKey2", 10);
+    entity1.addInfo(infoMap1);
+
+    // add event.
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId("end_event");
+    event11.setTimestamp(ts);
+    entity1.addEvent(event11);
+    TimelineEvent event12 = new TimelineEvent();
+    event12.setId("update_event");
+    event12.setTimestamp(ts - 10);
+    entity1.addEvent(event12);
+
+
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet1 = new HashSet<>();
+    isRelatedToSet1.add("relatedto3");
+    isRelatedToSet1.add("relatedto5");
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<>();
+    isRelatedTo1.put("task1", isRelatedToSet1);
+    Set<String> isRelatedToSet11 = new HashSet<>();
+    isRelatedToSet11.add("relatedto4");
+    isRelatedTo1.put("task2", isRelatedToSet11);
+    entity1.setIsRelatedToEntities(isRelatedTo1);
+
+    // add the relatesTo info
+    Set<String> relatesToSet1 = new HashSet<String>();
+    relatesToSet1.add("relatesto1");
+    relatesToSet1.add("relatesto2");
+    Map<String, Set<String>> relatesTo1 = new HashMap<>();
+    relatesTo1.put("container", relatesToSet1);
+    entity1.setRelatesToEntities(relatesTo1);
+
+    // add some config entries
+    Map<String, String> conf1 = new HashMap<>();
+    conf1.put("cfg_param1", "value1");
+    conf1.put("cfg_param2", "value2");
+    entity1.addConfigs(conf1);
+
+    // add metrics
+    Set<TimelineMetric> metrics1 = new HashSet<>();
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId("MAP1_SLOT_MILLIS");
+    Map<Long, Number> metricValues1 = new HashMap<>();
+    long ts1 = System.currentTimeMillis();
+    metricValues1.put(ts1 - 120000, 100000000);
+    metricValues1.put(ts1 - 100000, 200000000);
+    metricValues1.put(ts1 - 80000, 300000000);
+    metricValues1.put(ts1 - 60000, 400000000);
+    metricValues1.put(ts1 - 40000, 50000000000L);
+    metricValues1.put(ts1 - 20000, 60000000000L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues1);
+    metrics1.add(m2);
+    entity1.addMetrics(metrics1);
+    te.addEntity(entity1);
+
+    TimelineEntity entity2 = new TimelineEntity();
+    String id2 = "hello2";
+    entity2.setId(id2);
+    entity2.setType(type);
+    entity2.setCreatedTime(cTime + 40L);
+    TimelineEvent event21 = new TimelineEvent();
+    event21.setId("update_event");
+    event21.setTimestamp(ts - 20);
+    entity2.addEvent(event21);
+    Set<String> isRelatedToSet2 = new HashSet<>();
+    isRelatedToSet2.add("relatedto3");
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<>();
+    isRelatedTo2.put("task1", isRelatedToSet2);
+    entity2.setIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo3 = new HashMap<>();
+    Set<String> relatesToSet14 = new HashSet<>();
+    relatesToSet14.add("relatesto7");
+    relatesTo3.put("container2", relatesToSet14);
+    entity2.setRelatesToEntities(relatesTo3);
+    te.addEntity(entity2);
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
+      hbi.init(util.getConfiguration());
+      hbi.start();
+      String cluster = "cluster1";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      String appName = "application_1231111111_1111";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.stop();
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+}


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


[35/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js
new file mode 100644
index 0000000..61acb9b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js
@@ -0,0 +1,157 @@
+/*
+ * File:        jquery.dataTables.min.js
+ * Version:     1.9.4
+ * Author:      Allan Jardine (www.sprymedia.co.uk)
+ * Info:        www.datatables.net
+ *
+ * Copyright 2008-2012 Allan Jardine, all rights reserved.
+ *
+ * This source file is free software, under either the GPL v2 license or a
+ * BSD style license, available at:
+ *   http://datatables.net/license_gpl2
+ *   http://datatables.net/license_bsd
+ *
+ * This source file is distributed in the hope that it will be useful, but
+ * WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY
+ * or FITNESS FOR A PARTICULAR PURPOSE. See the license files for details.
+ */
+(function(la,s,p){(function(i){if(typeof define==="function"&&define.amd)define(["jquery"],i);else jQuery&&!jQuery.fn.dataTable&&i(jQuery)})(function(i){var l=function(h){function n(a,b){var c=l.defaults.columns,d=a.aoColumns.length;b=i.extend({},l.models.oColumn,c,{sSortingClass:a.oClasses.sSortable,sSortingClassJUI:a.oClasses.sSortJUI,nTh:b?b:s.createElement("th"),sTitle:c.sTitle?c.sTitle:b?b.innerHTML:"",aDataSort:c.aDataSort?c.aDataSort:[d],mData:c.mData?c.oDefaults:d});a.aoColumns.push(b);if(a.aoPreSearchCols[d]===
+p||a.aoPreSearchCols[d]===null)a.aoPreSearchCols[d]=i.extend({},l.models.oSearch);else{b=a.aoPreSearchCols[d];if(b.bRegex===p)b.bRegex=true;if(b.bSmart===p)b.bSmart=true;if(b.bCaseInsensitive===p)b.bCaseInsensitive=true}q(a,d,null)}function q(a,b,c){var d=a.aoColumns[b];if(c!==p&&c!==null){if(c.mDataProp&&!c.mData)c.mData=c.mDataProp;if(c.sType!==p){d.sType=c.sType;d._bAutoType=false}i.extend(d,c);r(d,c,"sWidth","sWidthOrig");if(c.iDataSort!==p)d.aDataSort=[c.iDataSort];r(d,c,"aDataSort")}var e=d.mRender?
+ca(d.mRender):null,f=ca(d.mData);d.fnGetData=function(g,j){var k=f(g,j);if(d.mRender&&j&&j!=="")return e(k,j,g);return k};d.fnSetData=Ja(d.mData);if(!a.oFeatures.bSort)d.bSortable=false;if(!d.bSortable||i.inArray("asc",d.asSorting)==-1&&i.inArray("desc",d.asSorting)==-1){d.sSortingClass=a.oClasses.sSortableNone;d.sSortingClassJUI=""}else if(i.inArray("asc",d.asSorting)==-1&&i.inArray("desc",d.asSorting)==-1){d.sSortingClass=a.oClasses.sSortable;d.sSortingClassJUI=a.oClasses.sSortJUI}else if(i.inArray("asc",
+d.asSorting)!=-1&&i.inArray("desc",d.asSorting)==-1){d.sSortingClass=a.oClasses.sSortableAsc;d.sSortingClassJUI=a.oClasses.sSortJUIAscAllowed}else if(i.inArray("asc",d.asSorting)==-1&&i.inArray("desc",d.asSorting)!=-1){d.sSortingClass=a.oClasses.sSortableDesc;d.sSortingClassJUI=a.oClasses.sSortJUIDescAllowed}}function o(a){if(a.oFeatures.bAutoWidth===false)return false;ta(a);for(var b=0,c=a.aoColumns.length;b<c;b++)a.aoColumns[b].nTh.style.width=a.aoColumns[b].sWidth}function v(a,b){a=A(a,"bVisible");
+return typeof a[b]==="number"?a[b]:null}function w(a,b){a=A(a,"bVisible");b=i.inArray(b,a);return b!==-1?b:null}function D(a){return A(a,"bVisible").length}function A(a,b){var c=[];i.map(a.aoColumns,function(d,e){d[b]&&c.push(e)});return c}function G(a){for(var b=l.ext.aTypes,c=b.length,d=0;d<c;d++){var e=b[d](a);if(e!==null)return e}return"string"}function E(a,b){b=b.split(",");for(var c=[],d=0,e=a.aoColumns.length;d<e;d++)for(var f=0;f<e;f++)if(a.aoColumns[d].sName==b[f]){c.push(f);break}return c}
+function Y(a){for(var b="",c=0,d=a.aoColumns.length;c<d;c++)b+=a.aoColumns[c].sName+",";if(b.length==d)return"";return b.slice(0,-1)}function ma(a,b,c,d){var e,f,g,j,k;if(b)for(e=b.length-1;e>=0;e--){var m=b[e].aTargets;i.isArray(m)||O(a,1,"aTargets must be an array of targets, not a "+typeof m);f=0;for(g=m.length;f<g;f++)if(typeof m[f]==="number"&&m[f]>=0){for(;a.aoColumns.length<=m[f];)n(a);d(m[f],b[e])}else if(typeof m[f]==="number"&&m[f]<0)d(a.aoColumns.length+m[f],b[e]);else if(typeof m[f]===
+"string"){j=0;for(k=a.aoColumns.length;j<k;j++)if(m[f]=="_all"||i(a.aoColumns[j].nTh).hasClass(m[f]))d(j,b[e])}}if(c){e=0;for(a=c.length;e<a;e++)d(e,c[e])}}function R(a,b){var c;c=i.isArray(b)?b.slice():i.extend(true,{},b);b=a.aoData.length;var d=i.extend(true,{},l.models.oRow);d._aData=c;a.aoData.push(d);var e;d=0;for(var f=a.aoColumns.length;d<f;d++){c=a.aoColumns[d];typeof c.fnRender==="function"&&c.bUseRendered&&c.mData!==null?S(a,b,d,da(a,b,d)):S(a,b,d,F(a,b,d));if(c._bAutoType&&c.sType!="string"){e=
+F(a,b,d,"type");if(e!==null&&e!==""){e=G(e);if(c.sType===null)c.sType=e;else if(c.sType!=e&&c.sType!="html")c.sType="string"}}}a.aiDisplayMaster.push(b);a.oFeatures.bDeferRender||ua(a,b);return b}function ea(a){var b,c,d,e,f,g,j;if(a.bDeferLoading||a.sAjaxSource===null)for(b=a.nTBody.firstChild;b;){if(b.nodeName.toUpperCase()=="TR"){c=a.aoData.length;b._DT_RowIndex=c;a.aoData.push(i.extend(true,{},l.models.oRow,{nTr:b}));a.aiDisplayMaster.push(c);f=b.firstChild;for(d=0;f;){g=f.nodeName.toUpperCase();
+if(g=="TD"||g=="TH"){S(a,c,d,i.trim(f.innerHTML));d++}f=f.nextSibling}}b=b.nextSibling}e=fa(a);d=[];b=0;for(c=e.length;b<c;b++)for(f=e[b].firstChild;f;){g=f.nodeName.toUpperCase();if(g=="TD"||g=="TH")d.push(f);f=f.nextSibling}c=0;for(e=a.aoColumns.length;c<e;c++){j=a.aoColumns[c];if(j.sTitle===null)j.sTitle=j.nTh.innerHTML;var k=j._bAutoType,m=typeof j.fnRender==="function",u=j.sClass!==null,x=j.bVisible,y,B;if(k||m||u||!x){g=0;for(b=a.aoData.length;g<b;g++){f=a.aoData[g];y=d[g*e+c];if(k&&j.sType!=
+"string"){B=F(a,g,c,"type");if(B!==""){B=G(B);if(j.sType===null)j.sType=B;else if(j.sType!=B&&j.sType!="html")j.sType="string"}}if(j.mRender)y.innerHTML=F(a,g,c,"display");else if(j.mData!==c)y.innerHTML=F(a,g,c,"display");if(m){B=da(a,g,c);y.innerHTML=B;j.bUseRendered&&S(a,g,c,B)}if(u)y.className+=" "+j.sClass;if(x)f._anHidden[c]=null;else{f._anHidden[c]=y;y.parentNode.removeChild(y)}j.fnCreatedCell&&j.fnCreatedCell.call(a.oInstance,y,F(a,g,c,"display"),f._aData,g,c)}}}if(a.aoRowCreatedCallback.length!==
+0){b=0;for(c=a.aoData.length;b<c;b++){f=a.aoData[b];K(a,"aoRowCreatedCallback",null,[f.nTr,f._aData,b])}}}function V(a,b){return b._DT_RowIndex!==p?b._DT_RowIndex:null}function va(a,b,c){b=W(a,b);var d=0;for(a=a.aoColumns.length;d<a;d++)if(b[d]===c)return d;return-1}function na(a,b,c,d){for(var e=[],f=0,g=d.length;f<g;f++)e.push(F(a,b,d[f],c));return e}function F(a,b,c,d){var e=a.aoColumns[c];if((c=e.fnGetData(a.aoData[b]._aData,d))===p){if(a.iDrawError!=a.iDraw&&e.sDefaultContent===null){O(a,0,"Requested unknown parameter "+
+(typeof e.mData=="function"?"{mData function}":"'"+e.mData+"'")+" from the data source for row "+b);a.iDrawError=a.iDraw}return e.sDefaultContent}if(c===null&&e.sDefaultContent!==null)c=e.sDefaultContent;else if(typeof c==="function")return c();if(d=="display"&&c===null)return"";return c}function S(a,b,c,d){a.aoColumns[c].fnSetData(a.aoData[b]._aData,d)}function ca(a){if(a===null)return function(){return null};else if(typeof a==="function")return function(c,d,e){return a(c,d,e)};else if(typeof a===
+"string"&&(a.indexOf(".")!==-1||a.indexOf("[")!==-1)){var b=function(c,d,e){var f=e.split("."),g;if(e!==""){var j=0;for(g=f.length;j<g;j++){if(e=f[j].match(ga)){f[j]=f[j].replace(ga,"");if(f[j]!=="")c=c[f[j]];g=[];f.splice(0,j+1);f=f.join(".");j=0;for(var k=c.length;j<k;j++)g.push(b(c[j],d,f));c=e[0].substring(1,e[0].length-1);c=c===""?g:g.join(c);break}if(c===null||c[f[j]]===p)return p;c=c[f[j]]}}return c};return function(c,d){return b(c,d,a)}}else return function(c){return c[a]}}function Ja(a){if(a===
+null)return function(){};else if(typeof a==="function")return function(c,d){a(c,"set",d)};else if(typeof a==="string"&&(a.indexOf(".")!==-1||a.indexOf("[")!==-1)){var b=function(c,d,e){e=e.split(".");var f,g,j=0;for(g=e.length-1;j<g;j++){if(f=e[j].match(ga)){e[j]=e[j].replace(ga,"");c[e[j]]=[];f=e.slice();f.splice(0,j+1);g=f.join(".");for(var k=0,m=d.length;k<m;k++){f={};b(f,d[k],g);c[e[j]].push(f)}return}if(c[e[j]]===null||c[e[j]]===p)c[e[j]]={};c=c[e[j]]}c[e[e.length-1].replace(ga,"")]=d};return function(c,
+d){return b(c,d,a)}}else return function(c,d){c[a]=d}}function oa(a){for(var b=[],c=a.aoData.length,d=0;d<c;d++)b.push(a.aoData[d]._aData);return b}function wa(a){a.aoData.splice(0,a.aoData.length);a.aiDisplayMaster.splice(0,a.aiDisplayMaster.length);a.aiDisplay.splice(0,a.aiDisplay.length);I(a)}function xa(a,b){for(var c=-1,d=0,e=a.length;d<e;d++)if(a[d]==b)c=d;else a[d]>b&&a[d]--;c!=-1&&a.splice(c,1)}function da(a,b,c){var d=a.aoColumns[c];return d.fnRender({iDataRow:b,iDataColumn:c,oSettings:a,
+aData:a.aoData[b]._aData,mDataProp:d.mData},F(a,b,c,"display"))}function ua(a,b){var c=a.aoData[b],d;if(c.nTr===null){c.nTr=s.createElement("tr");c.nTr._DT_RowIndex=b;if(c._aData.DT_RowId)c.nTr.id=c._aData.DT_RowId;if(c._aData.DT_RowClass)c.nTr.className=c._aData.DT_RowClass;for(var e=0,f=a.aoColumns.length;e<f;e++){var g=a.aoColumns[e];d=s.createElement(g.sCellType);d.innerHTML=typeof g.fnRender==="function"&&(!g.bUseRendered||g.mData===null)?da(a,b,e):F(a,b,e,"display");if(g.sClass!==null)d.className=
+g.sClass;if(g.bVisible){c.nTr.appendChild(d);c._anHidden[e]=null}else c._anHidden[e]=d;g.fnCreatedCell&&g.fnCreatedCell.call(a.oInstance,d,F(a,b,e,"display"),c._aData,b,e)}K(a,"aoRowCreatedCallback",null,[c.nTr,c._aData,b])}}function Ka(a){var b,c,d;if(i("th, td",a.nTHead).length!==0){b=0;for(d=a.aoColumns.length;b<d;b++){c=a.aoColumns[b].nTh;c.setAttribute("role","columnheader");if(a.aoColumns[b].bSortable){c.setAttribute("tabindex",a.iTabIndex);c.setAttribute("aria-controls",a.sTableId)}a.aoColumns[b].sClass!==
+null&&i(c).addClass(a.aoColumns[b].sClass);if(a.aoColumns[b].sTitle!=c.innerHTML)c.innerHTML=a.aoColumns[b].sTitle}}else{var e=s.createElement("tr");b=0;for(d=a.aoColumns.length;b<d;b++){c=a.aoColumns[b].nTh;c.innerHTML=a.aoColumns[b].sTitle;c.setAttribute("tabindex","0");a.aoColumns[b].sClass!==null&&i(c).addClass(a.aoColumns[b].sClass);e.appendChild(c)}i(a.nTHead).html("")[0].appendChild(e);ha(a.aoHeader,a.nTHead)}i(a.nTHead).children("tr").attr("role","row");if(a.bJUI){b=0;for(d=a.aoColumns.length;b<
+d;b++){c=a.aoColumns[b].nTh;e=s.createElement("div");e.className=a.oClasses.sSortJUIWrapper;i(c).contents().appendTo(e);var f=s.createElement("span");f.className=a.oClasses.sSortIcon;e.appendChild(f);c.appendChild(e)}}if(a.oFeatures.bSort)for(b=0;b<a.aoColumns.length;b++)a.aoColumns[b].bSortable!==false?ya(a,a.aoColumns[b].nTh,b):i(a.aoColumns[b].nTh).addClass(a.oClasses.sSortableNone);a.oClasses.sFooterTH!==""&&i(a.nTFoot).children("tr").children("th").addClass(a.oClasses.sFooterTH);if(a.nTFoot!==
+null){c=Z(a,null,a.aoFooter);b=0;for(d=a.aoColumns.length;b<d;b++)if(c[b]){a.aoColumns[b].nTf=c[b];a.aoColumns[b].sClass&&i(c[b]).addClass(a.aoColumns[b].sClass)}}}function ia(a,b,c){var d,e,f,g=[],j=[],k=a.aoColumns.length,m;if(c===p)c=false;d=0;for(e=b.length;d<e;d++){g[d]=b[d].slice();g[d].nTr=b[d].nTr;for(f=k-1;f>=0;f--)!a.aoColumns[f].bVisible&&!c&&g[d].splice(f,1);j.push([])}d=0;for(e=g.length;d<e;d++){if(a=g[d].nTr)for(;f=a.firstChild;)a.removeChild(f);f=0;for(b=g[d].length;f<b;f++){m=k=1;
+if(j[d][f]===p){a.appendChild(g[d][f].cell);for(j[d][f]=1;g[d+k]!==p&&g[d][f].cell==g[d+k][f].cell;){j[d+k][f]=1;k++}for(;g[d][f+m]!==p&&g[d][f].cell==g[d][f+m].cell;){for(c=0;c<k;c++)j[d+c][f+m]=1;m++}g[d][f].cell.rowSpan=k;g[d][f].cell.colSpan=m}}}}function H(a){var b=K(a,"aoPreDrawCallback","preDraw",[a]);if(i.inArray(false,b)!==-1)P(a,false);else{var c,d;b=[];var e=0,f=a.asStripeClasses.length;c=a.aoOpenRows.length;a.bDrawing=true;if(a.iInitDisplayStart!==p&&a.iInitDisplayStart!=-1){a._iDisplayStart=
+a.oFeatures.bServerSide?a.iInitDisplayStart:a.iInitDisplayStart>=a.fnRecordsDisplay()?0:a.iInitDisplayStart;a.iInitDisplayStart=-1;I(a)}if(a.bDeferLoading){a.bDeferLoading=false;a.iDraw++}else if(a.oFeatures.bServerSide){if(!a.bDestroying&&!La(a))return}else a.iDraw++;if(a.aiDisplay.length!==0){var g=a._iDisplayStart;d=a._iDisplayEnd;if(a.oFeatures.bServerSide){g=0;d=a.aoData.length}for(g=g;g<d;g++){var j=a.aoData[a.aiDisplay[g]];j.nTr===null&&ua(a,a.aiDisplay[g]);var k=j.nTr;if(f!==0){var m=a.asStripeClasses[e%
+f];if(j._sRowStripe!=m){i(k).removeClass(j._sRowStripe).addClass(m);j._sRowStripe=m}}K(a,"aoRowCallback",null,[k,a.aoData[a.aiDisplay[g]]._aData,e,g]);b.push(k);e++;if(c!==0)for(j=0;j<c;j++)if(k==a.aoOpenRows[j].nParent){b.push(a.aoOpenRows[j].nTr);break}}}else{b[0]=s.createElement("tr");if(a.asStripeClasses[0])b[0].className=a.asStripeClasses[0];c=a.oLanguage;f=c.sZeroRecords;if(a.iDraw==1&&a.sAjaxSource!==null&&!a.oFeatures.bServerSide)f=c.sLoadingRecords;else if(c.sEmptyTable&&a.fnRecordsTotal()===
+0)f=c.sEmptyTable;c=s.createElement("td");c.setAttribute("valign","top");c.colSpan=D(a);c.className=a.oClasses.sRowEmpty;c.innerHTML=za(a,f);b[e].appendChild(c)}K(a,"aoHeaderCallback","header",[i(a.nTHead).children("tr")[0],oa(a),a._iDisplayStart,a.fnDisplayEnd(),a.aiDisplay]);K(a,"aoFooterCallback","footer",[i(a.nTFoot).children("tr")[0],oa(a),a._iDisplayStart,a.fnDisplayEnd(),a.aiDisplay]);e=s.createDocumentFragment();c=s.createDocumentFragment();if(a.nTBody){f=a.nTBody.parentNode;c.appendChild(a.nTBody);
+if(!a.oScroll.bInfinite||!a._bInitComplete||a.bSorted||a.bFiltered)for(;c=a.nTBody.firstChild;)a.nTBody.removeChild(c);c=0;for(d=b.length;c<d;c++)e.appendChild(b[c]);a.nTBody.appendChild(e);f!==null&&f.appendChild(a.nTBody)}K(a,"aoDrawCallback","draw",[a]);a.bSorted=false;a.bFiltered=false;a.bDrawing=false;if(a.oFeatures.bServerSide){P(a,false);a._bInitComplete||pa(a)}}}function qa(a){if(a.oFeatures.bSort)$(a,a.oPreviousSearch);else if(a.oFeatures.bFilter)X(a,a.oPreviousSearch);else{I(a);H(a)}}function Ma(a){var b=
+i("<div></div>")[0];a.nTable.parentNode.insertBefore(b,a.nTable);a.nTableWrapper=i('<div id="'+a.sTableId+'_wrapper" class="'+a.oClasses.sWrapper+'" role="grid"></div>')[0];a.nTableReinsertBefore=a.nTable.nextSibling;for(var c=a.nTableWrapper,d=a.sDom.split(""),e,f,g,j,k,m,u,x=0;x<d.length;x++){f=0;g=d[x];if(g=="<"){j=i("<div></div>")[0];k=d[x+1];if(k=="'"||k=='"'){m="";for(u=2;d[x+u]!=k;){m+=d[x+u];u++}if(m=="H")m=a.oClasses.sJUIHeader;else if(m=="F")m=a.oClasses.sJUIFooter;if(m.indexOf(".")!=-1){k=
+m.split(".");j.id=k[0].substr(1,k[0].length-1);j.className=k[1]}else if(m.charAt(0)=="#")j.id=m.substr(1,m.length-1);else j.className=m;x+=u}c.appendChild(j);c=j}else if(g==">")c=c.parentNode;else if(g=="l"&&a.oFeatures.bPaginate&&a.oFeatures.bLengthChange){e=Na(a);f=1}else if(g=="f"&&a.oFeatures.bFilter){e=Oa(a);f=1}else if(g=="r"&&a.oFeatures.bProcessing){e=Pa(a);f=1}else if(g=="t"){e=Qa(a);f=1}else if(g=="i"&&a.oFeatures.bInfo){e=Ra(a);f=1}else if(g=="p"&&a.oFeatures.bPaginate){e=Sa(a);f=1}else if(l.ext.aoFeatures.length!==
+0){j=l.ext.aoFeatures;u=0;for(k=j.length;u<k;u++)if(g==j[u].cFeature){if(e=j[u].fnInit(a))f=1;break}}if(f==1&&e!==null){if(typeof a.aanFeatures[g]!=="object")a.aanFeatures[g]=[];a.aanFeatures[g].push(e);c.appendChild(e)}}b.parentNode.replaceChild(a.nTableWrapper,b)}function ha(a,b){b=i(b).children("tr");var c,d,e,f,g,j,k,m,u,x,y=function(B,T,M){for(B=B[T];B[M];)M++;return M};a.splice(0,a.length);e=0;for(j=b.length;e<j;e++)a.push([]);e=0;for(j=b.length;e<j;e++){c=b[e];for(d=c.firstChild;d;){if(d.nodeName.toUpperCase()==
+"TD"||d.nodeName.toUpperCase()=="TH"){m=d.getAttribute("colspan")*1;u=d.getAttribute("rowspan")*1;m=!m||m===0||m===1?1:m;u=!u||u===0||u===1?1:u;k=y(a,e,0);x=m===1?true:false;for(g=0;g<m;g++)for(f=0;f<u;f++){a[e+f][k+g]={cell:d,unique:x};a[e+f].nTr=c}}d=d.nextSibling}}}function Z(a,b,c){var d=[];if(!c){c=a.aoHeader;if(b){c=[];ha(c,b)}}b=0;for(var e=c.length;b<e;b++)for(var f=0,g=c[b].length;f<g;f++)if(c[b][f].unique&&(!d[f]||!a.bSortCellsTop))d[f]=c[b][f].cell;return d}function La(a){if(a.bAjaxDataGet){a.iDraw++;
+P(a,true);var b=Ta(a);Aa(a,b);a.fnServerData.call(a.oInstance,a.sAjaxSource,b,function(c){Ua(a,c)},a);return false}else return true}function Ta(a){var b=a.aoColumns.length,c=[],d,e,f,g;c.push({name:"sEcho",value:a.iDraw});c.push({name:"iColumns",value:b});c.push({name:"sColumns",value:Y(a)});c.push({name:"iDisplayStart",value:a._iDisplayStart});c.push({name:"iDisplayLength",value:a.oFeatures.bPaginate!==false?a._iDisplayLength:-1});for(f=0;f<b;f++){d=a.aoColumns[f].mData;c.push({name:"mDataProp_"+
+f,value:typeof d==="function"?"function":d})}if(a.oFeatures.bFilter!==false){c.push({name:"sSearch",value:a.oPreviousSearch.sSearch});c.push({name:"bRegex",value:a.oPreviousSearch.bRegex});for(f=0;f<b;f++){c.push({name:"sSearch_"+f,value:a.aoPreSearchCols[f].sSearch});c.push({name:"bRegex_"+f,value:a.aoPreSearchCols[f].bRegex});c.push({name:"bSearchable_"+f,value:a.aoColumns[f].bSearchable})}}if(a.oFeatures.bSort!==false){var j=0;d=a.aaSortingFixed!==null?a.aaSortingFixed.concat(a.aaSorting):a.aaSorting.slice();
+for(f=0;f<d.length;f++){e=a.aoColumns[d[f][0]].aDataSort;for(g=0;g<e.length;g++){c.push({name:"iSortCol_"+j,value:e[g]});c.push({name:"sSortDir_"+j,value:d[f][1]});j++}}c.push({name:"iSortingCols",value:j});for(f=0;f<b;f++)c.push({name:"bSortable_"+f,value:a.aoColumns[f].bSortable})}return c}function Aa(a,b){K(a,"aoServerParams","serverParams",[b])}function Ua(a,b){if(b.sEcho!==p)if(b.sEcho*1<a.iDraw)return;else a.iDraw=b.sEcho*1;if(!a.oScroll.bInfinite||a.oScroll.bInfinite&&(a.bSorted||a.bFiltered))wa(a);
+a._iRecordsTotal=parseInt(b.iTotalRecords,10);a._iRecordsDisplay=parseInt(b.iTotalDisplayRecords,10);var c=Y(a);c=b.sColumns!==p&&c!==""&&b.sColumns!=c;var d;if(c)d=E(a,b.sColumns);b=ca(a.sAjaxDataProp)(b);for(var e=0,f=b.length;e<f;e++)if(c){for(var g=[],j=0,k=a.aoColumns.length;j<k;j++)g.push(b[e][d[j]]);R(a,g)}else R(a,b[e]);a.aiDisplay=a.aiDisplayMaster.slice();a.bAjaxDataGet=false;H(a);a.bAjaxDataGet=true;P(a,false)}function Oa(a){var b=a.oPreviousSearch,c=a.oLanguage.sSearch;c=c.indexOf("_INPUT_")!==
+-1?c.replace("_INPUT_",'<input type="text" />'):c===""?'<input type="text" />':c+' <input type="text" />';var d=s.createElement("div");d.className=a.oClasses.sFilter;d.innerHTML="<label>"+c+"</label>";if(!a.aanFeatures.f)d.id=a.sTableId+"_filter";c=i('input[type="text"]',d);d._DT_Input=c[0];c.val(b.sSearch.replace('"',"&quot;"));c.bind("keyup.DT",function(){for(var e=a.aanFeatures.f,f=this.value===""?"":this.value,g=0,j=e.length;g<j;g++)e[g]!=i(this).parents("div.dataTables_filter")[0]&&i(e[g]._DT_Input).val(f);
+f!=b.sSearch&&X(a,{sSearch:f,bRegex:b.bRegex,bSmart:b.bSmart,bCaseInsensitive:b.bCaseInsensitive})});c.attr("aria-controls",a.sTableId).bind("keypress.DT",function(e){if(e.keyCode==13)return false});return d}function X(a,b,c){var d=a.oPreviousSearch,e=a.aoPreSearchCols,f=function(g){d.sSearch=g.sSearch;d.bRegex=g.bRegex;d.bSmart=g.bSmart;d.bCaseInsensitive=g.bCaseInsensitive};if(a.oFeatures.bServerSide)f(b);else{Va(a,b.sSearch,c,b.bRegex,b.bSmart,b.bCaseInsensitive);f(b);for(b=0;b<a.aoPreSearchCols.length;b++)Wa(a,
+e[b].sSearch,b,e[b].bRegex,e[b].bSmart,e[b].bCaseInsensitive);Xa(a)}a.bFiltered=true;i(a.oInstance).trigger("filter",a);a._iDisplayStart=0;I(a);H(a);Ba(a,0)}function Xa(a){for(var b=l.ext.afnFiltering,c=A(a,"bSearchable"),d=0,e=b.length;d<e;d++)for(var f=0,g=0,j=a.aiDisplay.length;g<j;g++){var k=a.aiDisplay[g-f];if(!b[d](a,na(a,k,"filter",c),k)){a.aiDisplay.splice(g-f,1);f++}}}function Wa(a,b,c,d,e,f){if(b!==""){var g=0;b=Ca(b,d,e,f);for(d=a.aiDisplay.length-1;d>=0;d--){e=Ya(F(a,a.aiDisplay[d],c,
+"filter"),a.aoColumns[c].sType);if(!b.test(e)){a.aiDisplay.splice(d,1);g++}}}}function Va(a,b,c,d,e,f){d=Ca(b,d,e,f);e=a.oPreviousSearch;c||(c=0);if(l.ext.afnFiltering.length!==0)c=1;if(b.length<=0){a.aiDisplay.splice(0,a.aiDisplay.length);a.aiDisplay=a.aiDisplayMaster.slice()}else if(a.aiDisplay.length==a.aiDisplayMaster.length||e.sSearch.length>b.length||c==1||b.indexOf(e.sSearch)!==0){a.aiDisplay.splice(0,a.aiDisplay.length);Ba(a,1);for(b=0;b<a.aiDisplayMaster.length;b++)d.test(a.asDataSearch[b])&&
+a.aiDisplay.push(a.aiDisplayMaster[b])}else for(b=c=0;b<a.asDataSearch.length;b++)if(!d.test(a.asDataSearch[b])){a.aiDisplay.splice(b-c,1);c++}}function Ba(a,b){if(!a.oFeatures.bServerSide){a.asDataSearch=[];var c=A(a,"bSearchable");b=b===1?a.aiDisplayMaster:a.aiDisplay;for(var d=0,e=b.length;d<e;d++)a.asDataSearch[d]=Da(a,na(a,b[d],"filter",c))}}function Da(a,b){a=b.join("  ");if(a.indexOf("&")!==-1)a=i("<div>").html(a).text();return a.replace(/[\n\r]/g," ")}function Ca(a,b,c,d){if(c){a=b?a.split(" "):
+Ea(a).split(" ");a="^(?=.*?"+a.join(")(?=.*?")+").*$";return new RegExp(a,d?"i":"")}else{a=b?a:Ea(a);return new RegExp(a,d?"i":"")}}function Ya(a,b){if(typeof l.ext.ofnSearch[b]==="function")return l.ext.ofnSearch[b](a);else if(a===null)return"";else if(b=="html")return a.replace(/[\r\n]/g," ").replace(/<.*?>/g,"");else if(typeof a==="string")return a.replace(/[\r\n]/g," ");return a}function Ea(a){return a.replace(new RegExp("(\\/|\\.|\\*|\\+|\\?|\\||\\(|\\)|\\[|\\]|\\{|\\}|\\\\|\\$|\\^|\\-)","g"),
+"\\$1")}function Ra(a){var b=s.createElement("div");b.className=a.oClasses.sInfo;if(!a.aanFeatures.i){a.aoDrawCallback.push({fn:Za,sName:"information"});b.id=a.sTableId+"_info"}a.nTable.setAttribute("aria-describedby",a.sTableId+"_info");return b}function Za(a){if(!(!a.oFeatures.bInfo||a.aanFeatures.i.length===0)){var b=a.oLanguage,c=a._iDisplayStart+1,d=a.fnDisplayEnd(),e=a.fnRecordsTotal(),f=a.fnRecordsDisplay(),g;g=f===0?b.sInfoEmpty:b.sInfo;if(f!=e)g+=" "+b.sInfoFiltered;g+=b.sInfoPostFix;g=za(a,
+g);if(b.fnInfoCallback!==null)g=b.fnInfoCallback.call(a.oInstance,a,c,d,e,f,g);a=a.aanFeatures.i;b=0;for(c=a.length;b<c;b++)i(a[b]).html(g)}}function za(a,b){var c=a.fnFormatNumber(a._iDisplayStart+1),d=a.fnDisplayEnd();d=a.fnFormatNumber(d);var e=a.fnRecordsDisplay();e=a.fnFormatNumber(e);var f=a.fnRecordsTotal();f=a.fnFormatNumber(f);if(a.oScroll.bInfinite)c=a.fnFormatNumber(1);return b.replace(/_START_/g,c).replace(/_END_/g,d).replace(/_TOTAL_/g,e).replace(/_MAX_/g,f)}function ra(a){var b,c,d=
+a.iInitDisplayStart;if(a.bInitialised===false)setTimeout(function(){ra(a)},200);else{Ma(a);Ka(a);ia(a,a.aoHeader);a.nTFoot&&ia(a,a.aoFooter);P(a,true);a.oFeatures.bAutoWidth&&ta(a);b=0;for(c=a.aoColumns.length;b<c;b++)if(a.aoColumns[b].sWidth!==null)a.aoColumns[b].nTh.style.width=t(a.aoColumns[b].sWidth);if(a.oFeatures.bSort)$(a);else if(a.oFeatures.bFilter)X(a,a.oPreviousSearch);else{a.aiDisplay=a.aiDisplayMaster.slice();I(a);H(a)}if(a.sAjaxSource!==null&&!a.oFeatures.bServerSide){c=[];Aa(a,c);a.fnServerData.call(a.oInstance,
+a.sAjaxSource,c,function(e){var f=a.sAjaxDataProp!==""?ca(a.sAjaxDataProp)(e):e;for(b=0;b<f.length;b++)R(a,f[b]);a.iInitDisplayStart=d;if(a.oFeatures.bSort)$(a);else{a.aiDisplay=a.aiDisplayMaster.slice();I(a);H(a)}P(a,false);pa(a,e)},a)}else if(!a.oFeatures.bServerSide){P(a,false);pa(a)}}}function pa(a,b){a._bInitComplete=true;K(a,"aoInitComplete","init",[a,b])}function Fa(a){var b=l.defaults.oLanguage;!a.sEmptyTable&&a.sZeroRecords&&b.sEmptyTable==="No data available in table"&&r(a,a,"sZeroRecords",
+"sEmptyTable");!a.sLoadingRecords&&a.sZeroRecords&&b.sLoadingRecords==="Loading..."&&r(a,a,"sZeroRecords","sLoadingRecords")}function Na(a){if(a.oScroll.bInfinite)return null;var b='<select size="1" '+('name="'+a.sTableId+'_length"')+">",c,d,e=a.aLengthMenu;if(e.length==2&&typeof e[0]==="object"&&typeof e[1]==="object"){c=0;for(d=e[0].length;c<d;c++)b+='<option value="'+e[0][c]+'">'+e[1][c]+"</option>"}else{c=0;for(d=e.length;c<d;c++)b+='<option value="'+e[c]+'">'+e[c]+"</option>"}b+="</select>";
+e=s.createElement("div");if(!a.aanFeatures.l)e.id=a.sTableId+"_length";e.className=a.oClasses.sLength;e.innerHTML="<label>"+a.oLanguage.sLengthMenu.replace("_MENU_",b)+"</label>";i('select option[value="'+a._iDisplayLength+'"]',e).attr("selected",true);i("select",e).bind("change.DT",function(){var f=i(this).val(),g=a.aanFeatures.l;c=0;for(d=g.length;c<d;c++)g[c]!=this.parentNode&&i("select",g[c]).val(f);a._iDisplayLength=parseInt(f,10);I(a);if(a.fnDisplayEnd()==a.fnRecordsDisplay()){a._iDisplayStart=
+a.fnDisplayEnd()-a._iDisplayLength;if(a._iDisplayStart<0)a._iDisplayStart=0}if(a._iDisplayLength==-1)a._iDisplayStart=0;H(a)});i("select",e).attr("aria-controls",a.sTableId);return e}function I(a){a._iDisplayEnd=a.oFeatures.bPaginate===false?a.aiDisplay.length:a._iDisplayStart+a._iDisplayLength>a.aiDisplay.length||a._iDisplayLength==-1?a.aiDisplay.length:a._iDisplayStart+a._iDisplayLength}function Sa(a){if(a.oScroll.bInfinite)return null;var b=s.createElement("div");b.className=a.oClasses.sPaging+
+a.sPaginationType;l.ext.oPagination[a.sPaginationType].fnInit(a,b,function(c){I(c);H(c)});a.aanFeatures.p||a.aoDrawCallback.push({fn:function(c){l.ext.oPagination[c.sPaginationType].fnUpdate(c,function(d){I(d);H(d)})},sName:"pagination"});return b}function Ga(a,b){var c=a._iDisplayStart;if(typeof b==="number"){a._iDisplayStart=b*a._iDisplayLength;if(a._iDisplayStart>a.fnRecordsDisplay())a._iDisplayStart=0}else if(b=="first")a._iDisplayStart=0;else if(b=="previous"){a._iDisplayStart=a._iDisplayLength>=
+0?a._iDisplayStart-a._iDisplayLength:0;if(a._iDisplayStart<0)a._iDisplayStart=0}else if(b=="next")if(a._iDisplayLength>=0){if(a._iDisplayStart+a._iDisplayLength<a.fnRecordsDisplay())a._iDisplayStart+=a._iDisplayLength}else a._iDisplayStart=0;else if(b=="last")if(a._iDisplayLength>=0){b=parseInt((a.fnRecordsDisplay()-1)/a._iDisplayLength,10)+1;a._iDisplayStart=(b-1)*a._iDisplayLength}else a._iDisplayStart=0;else O(a,0,"Unknown paging action: "+b);i(a.oInstance).trigger("page",a);return c!=a._iDisplayStart}
+function Pa(a){var b=s.createElement("div");if(!a.aanFeatures.r)b.id=a.sTableId+"_processing";b.innerHTML=a.oLanguage.sProcessing;b.className=a.oClasses.sProcessing;a.nTable.parentNode.insertBefore(b,a.nTable);return b}function P(a,b){if(a.oFeatures.bProcessing)for(var c=a.aanFeatures.r,d=0,e=c.length;d<e;d++)c[d].style.visibility=b?"visible":"hidden";i(a.oInstance).trigger("processing",[a,b])}function Qa(a){if(a.oScroll.sX===""&&a.oScroll.sY==="")return a.nTable;var b=s.createElement("div"),c=s.createElement("div"),
+d=s.createElement("div"),e=s.createElement("div"),f=s.createElement("div"),g=s.createElement("div"),j=a.nTable.cloneNode(false),k=a.nTable.cloneNode(false),m=a.nTable.getElementsByTagName("thead")[0],u=a.nTable.getElementsByTagName("tfoot").length===0?null:a.nTable.getElementsByTagName("tfoot")[0],x=a.oClasses;c.appendChild(d);f.appendChild(g);e.appendChild(a.nTable);b.appendChild(c);b.appendChild(e);d.appendChild(j);j.appendChild(m);if(u!==null){b.appendChild(f);g.appendChild(k);k.appendChild(u)}b.className=
+x.sScrollWrapper;c.className=x.sScrollHead;d.className=x.sScrollHeadInner;e.className=x.sScrollBody;f.className=x.sScrollFoot;g.className=x.sScrollFootInner;if(a.oScroll.bAutoCss){c.style.overflow="hidden";c.style.position="relative";f.style.overflow="hidden";e.style.overflow="auto"}c.style.border="0";c.style.width="100%";f.style.border="0";d.style.width=a.oScroll.sXInner!==""?a.oScroll.sXInner:"100%";j.removeAttribute("id");j.style.marginLeft="0";a.nTable.style.marginLeft="0";if(u!==null){k.removeAttribute("id");
+k.style.marginLeft="0"}d=i(a.nTable).children("caption");if(d.length>0){d=d[0];if(d._captionSide==="top")j.appendChild(d);else d._captionSide==="bottom"&&u&&k.appendChild(d)}if(a.oScroll.sX!==""){c.style.width=t(a.oScroll.sX);e.style.width=t(a.oScroll.sX);if(u!==null)f.style.width=t(a.oScroll.sX);i(e).scroll(function(){c.scrollLeft=this.scrollLeft;if(u!==null)f.scrollLeft=this.scrollLeft})}if(a.oScroll.sY!=="")e.style.height=t(a.oScroll.sY);a.aoDrawCallback.push({fn:$a,sName:"scrolling"});a.oScroll.bInfinite&&
+i(e).scroll(function(){if(!a.bDrawing&&i(this).scrollTop()!==0)if(i(this).scrollTop()+i(this).height()>i(a.nTable).height()-a.oScroll.iLoadGap)if(a.fnDisplayEnd()<a.fnRecordsDisplay()){Ga(a,"next");I(a);H(a)}});a.nScrollHead=c;a.nScrollFoot=f;return b}function $a(a){var b=a.nScrollHead.getElementsByTagName("div")[0],c=b.getElementsByTagName("table")[0],d=a.nTable.parentNode,e,f,g,j,k,m,u,x,y=[],B=[],T=a.nTFoot!==null?a.nScrollFoot.getElementsByTagName("div")[0]:null,M=a.nTFoot!==null?T.getElementsByTagName("table")[0]:
+null,L=a.oBrowser.bScrollOversize,ja=function(z){u=z.style;u.paddingTop="0";u.paddingBottom="0";u.borderTopWidth="0";u.borderBottomWidth="0";u.height=0};i(a.nTable).children("thead, tfoot").remove();e=i(a.nTHead).clone()[0];a.nTable.insertBefore(e,a.nTable.childNodes[0]);g=a.nTHead.getElementsByTagName("tr");j=e.getElementsByTagName("tr");if(a.nTFoot!==null){k=i(a.nTFoot).clone()[0];a.nTable.insertBefore(k,a.nTable.childNodes[1]);m=a.nTFoot.getElementsByTagName("tr");k=k.getElementsByTagName("tr")}if(a.oScroll.sX===
+""){d.style.width="100%";b.parentNode.style.width="100%"}var U=Z(a,e);e=0;for(f=U.length;e<f;e++){x=v(a,e);U[e].style.width=a.aoColumns[x].sWidth}a.nTFoot!==null&&N(function(z){z.style.width=""},k);if(a.oScroll.bCollapse&&a.oScroll.sY!=="")d.style.height=d.offsetHeight+a.nTHead.offsetHeight+"px";e=i(a.nTable).outerWidth();if(a.oScroll.sX===""){a.nTable.style.width="100%";if(L&&(i("tbody",d).height()>d.offsetHeight||i(d).css("overflow-y")=="scroll"))a.nTable.style.width=t(i(a.nTable).outerWidth()-
+a.oScroll.iBarWidth)}else if(a.oScroll.sXInner!=="")a.nTable.style.width=t(a.oScroll.sXInner);else if(e==i(d).width()&&i(d).height()<i(a.nTable).height()){a.nTable.style.width=t(e-a.oScroll.iBarWidth);if(i(a.nTable).outerWidth()>e-a.oScroll.iBarWidth)a.nTable.style.width=t(e)}else a.nTable.style.width=t(e);e=i(a.nTable).outerWidth();N(ja,j);N(function(z){y.push(t(i(z).width()))},j);N(function(z,Q){z.style.width=y[Q]},g);i(j).height(0);if(a.nTFoot!==null){N(ja,k);N(function(z){B.push(t(i(z).width()))},
+k);N(function(z,Q){z.style.width=B[Q]},m);i(k).height(0)}N(function(z,Q){z.innerHTML="";z.style.width=y[Q]},j);a.nTFoot!==null&&N(function(z,Q){z.innerHTML="";z.style.width=B[Q]},k);if(i(a.nTable).outerWidth()<e){g=d.scrollHeight>d.offsetHeight||i(d).css("overflow-y")=="scroll"?e+a.oScroll.iBarWidth:e;if(L&&(d.scrollHeight>d.offsetHeight||i(d).css("overflow-y")=="scroll"))a.nTable.style.width=t(g-a.oScroll.iBarWidth);d.style.width=t(g);a.nScrollHead.style.width=t(g);if(a.nTFoot!==null)a.nScrollFoot.style.width=
+t(g);if(a.oScroll.sX==="")O(a,1,"The table cannot fit into the current element which will cause column misalignment. The table has been drawn at its minimum possible width.");else a.oScroll.sXInner!==""&&O(a,1,"The table cannot fit into the current element which will cause column misalignment. Increase the sScrollXInner value or remove it to allow automatic calculation")}else{d.style.width=t("100%");a.nScrollHead.style.width=t("100%");if(a.nTFoot!==null)a.nScrollFoot.style.width=t("100%")}if(a.oScroll.sY===
+"")if(L)d.style.height=t(a.nTable.offsetHeight+a.oScroll.iBarWidth);if(a.oScroll.sY!==""&&a.oScroll.bCollapse){d.style.height=t(a.oScroll.sY);L=a.oScroll.sX!==""&&a.nTable.offsetWidth>d.offsetWidth?a.oScroll.iBarWidth:0;if(a.nTable.offsetHeight<d.offsetHeight)d.style.height=t(a.nTable.offsetHeight+L)}L=i(a.nTable).outerWidth();c.style.width=t(L);b.style.width=t(L);c=i(a.nTable).height()>d.clientHeight||i(d).css("overflow-y")=="scroll";b.style.paddingRight=c?a.oScroll.iBarWidth+"px":"0px";if(a.nTFoot!==
+null){M.style.width=t(L);T.style.width=t(L);T.style.paddingRight=c?a.oScroll.iBarWidth+"px":"0px"}i(d).scroll();if(a.bSorted||a.bFiltered)d.scrollTop=0}function N(a,b,c){for(var d=0,e=0,f=b.length,g,j;e<f;){g=b[e].firstChild;for(j=c?c[e].firstChild:null;g;){if(g.nodeType===1){c?a(g,j,d):a(g,d);d++}g=g.nextSibling;j=c?j.nextSibling:null}e++}}function ab(a,b){if(!a||a===null||a==="")return 0;if(!b)b=s.body;var c=s.createElement("div");c.style.width=t(a);b.appendChild(c);a=c.offsetWidth;b.removeChild(c);
+return a}function ta(a){var b=0,c,d=0,e=a.aoColumns.length,f,g,j=i("th",a.nTHead),k=a.nTable.getAttribute("width");g=a.nTable.parentNode;for(f=0;f<e;f++)if(a.aoColumns[f].bVisible){d++;if(a.aoColumns[f].sWidth!==null){c=ab(a.aoColumns[f].sWidthOrig,g);if(c!==null)a.aoColumns[f].sWidth=t(c);b++}}if(e==j.length&&b===0&&d==e&&a.oScroll.sX===""&&a.oScroll.sY==="")for(f=0;f<a.aoColumns.length;f++){c=i(j[f]).width();if(c!==null)a.aoColumns[f].sWidth=t(c)}else{b=a.nTable.cloneNode(false);f=a.nTHead.cloneNode(true);
+d=s.createElement("tbody");c=s.createElement("tr");b.removeAttribute("id");b.appendChild(f);if(a.nTFoot!==null){b.appendChild(a.nTFoot.cloneNode(true));N(function(u){u.style.width=""},b.getElementsByTagName("tr"))}b.appendChild(d);d.appendChild(c);d=i("thead th",b);if(d.length===0)d=i("tbody tr:eq(0)>td",b);j=Z(a,f);for(f=d=0;f<e;f++){var m=a.aoColumns[f];if(m.bVisible&&m.sWidthOrig!==null&&m.sWidthOrig!=="")j[f-d].style.width=t(m.sWidthOrig);else if(m.bVisible)j[f-d].style.width="";else d++}for(f=
+0;f<e;f++)if(a.aoColumns[f].bVisible){d=bb(a,f);if(d!==null){d=d.cloneNode(true);if(a.aoColumns[f].sContentPadding!=="")d.innerHTML+=a.aoColumns[f].sContentPadding;c.appendChild(d)}}g.appendChild(b);if(a.oScroll.sX!==""&&a.oScroll.sXInner!=="")b.style.width=t(a.oScroll.sXInner);else if(a.oScroll.sX!==""){b.style.width="";if(i(b).width()<g.offsetWidth)b.style.width=t(g.offsetWidth)}else if(a.oScroll.sY!=="")b.style.width=t(g.offsetWidth);else if(k)b.style.width=t(k);b.style.visibility="hidden";cb(a,
+b);e=i("tbody tr:eq(0)",b).children();if(e.length===0)e=Z(a,i("thead",b)[0]);if(a.oScroll.sX!==""){for(f=d=g=0;f<a.aoColumns.length;f++)if(a.aoColumns[f].bVisible){g+=a.aoColumns[f].sWidthOrig===null?i(e[d]).outerWidth():parseInt(a.aoColumns[f].sWidth.replace("px",""),10)+(i(e[d]).outerWidth()-i(e[d]).width());d++}b.style.width=t(g);a.nTable.style.width=t(g)}for(f=d=0;f<a.aoColumns.length;f++)if(a.aoColumns[f].bVisible){g=i(e[d]).width();if(g!==null&&g>0)a.aoColumns[f].sWidth=t(g);d++}e=i(b).css("width");
+a.nTable.style.width=e.indexOf("%")!==-1?e:t(i(b).outerWidth());b.parentNode.removeChild(b)}if(k)a.nTable.style.width=t(k)}function cb(a,b){if(a.oScroll.sX===""&&a.oScroll.sY!==""){i(b).width();b.style.width=t(i(b).outerWidth()-a.oScroll.iBarWidth)}else if(a.oScroll.sX!=="")b.style.width=t(i(b).outerWidth())}function bb(a,b){var c=db(a,b);if(c<0)return null;if(a.aoData[c].nTr===null){var d=s.createElement("td");d.innerHTML=F(a,c,b,"");return d}return W(a,c)[b]}function db(a,b){for(var c=-1,d=-1,e=
+0;e<a.aoData.length;e++){var f=F(a,e,b,"display")+"";f=f.replace(/<.*?>/g,"");if(f.length>c){c=f.length;d=e}}return d}function t(a){if(a===null)return"0px";if(typeof a=="number"){if(a<0)return"0px";return a+"px"}var b=a.charCodeAt(a.length-1);if(b<48||b>57)return a;return a+"px"}function eb(){var a=s.createElement("p"),b=a.style;b.width="100%";b.height="200px";b.padding="0px";var c=s.createElement("div");b=c.style;b.position="absolute";b.top="0px";b.left="0px";b.visibility="hidden";b.width="200px";
+b.height="150px";b.padding="0px";b.overflow="hidden";c.appendChild(a);s.body.appendChild(c);b=a.offsetWidth;c.style.overflow="scroll";a=a.offsetWidth;if(b==a)a=c.clientWidth;s.body.removeChild(c);return b-a}function $(a,b){var c,d,e,f,g,j,k=[],m=[],u=l.ext.oSort,x=a.aoData,y=a.aoColumns,B=a.oLanguage.oAria;if(!a.oFeatures.bServerSide&&(a.aaSorting.length!==0||a.aaSortingFixed!==null)){k=a.aaSortingFixed!==null?a.aaSortingFixed.concat(a.aaSorting):a.aaSorting.slice();for(c=0;c<k.length;c++){d=k[c][0];
+e=w(a,d);f=a.aoColumns[d].sSortDataType;if(l.ext.afnSortData[f]){g=l.ext.afnSortData[f].call(a.oInstance,a,d,e);if(g.length===x.length){e=0;for(f=x.length;e<f;e++)S(a,e,d,g[e])}else O(a,0,"Returned data sort array (col "+d+") is the wrong length")}}c=0;for(d=a.aiDisplayMaster.length;c<d;c++)m[a.aiDisplayMaster[c]]=c;var T=k.length,M;c=0;for(d=x.length;c<d;c++)for(e=0;e<T;e++){M=y[k[e][0]].aDataSort;g=0;for(j=M.length;g<j;g++){f=y[M[g]].sType;f=u[(f?f:"string")+"-pre"];x[c]._aSortData[M[g]]=f?f(F(a,
+c,M[g],"sort")):F(a,c,M[g],"sort")}}a.aiDisplayMaster.sort(function(L,ja){var U,z,Q,aa,ka;for(U=0;U<T;U++){ka=y[k[U][0]].aDataSort;z=0;for(Q=ka.length;z<Q;z++){aa=y[ka[z]].sType;aa=u[(aa?aa:"string")+"-"+k[U][1]](x[L]._aSortData[ka[z]],x[ja]._aSortData[ka[z]]);if(aa!==0)return aa}}return u["numeric-asc"](m[L],m[ja])})}if((b===p||b)&&!a.oFeatures.bDeferRender)ba(a);c=0;for(d=a.aoColumns.length;c<d;c++){e=y[c].sTitle.replace(/<.*?>/g,"");b=y[c].nTh;b.removeAttribute("aria-sort");b.removeAttribute("aria-label");
+if(y[c].bSortable)if(k.length>0&&k[0][0]==c){b.setAttribute("aria-sort",k[0][1]=="asc"?"ascending":"descending");b.setAttribute("aria-label",e+((y[c].asSorting[k[0][2]+1]?y[c].asSorting[k[0][2]+1]:y[c].asSorting[0])=="asc"?B.sSortAscending:B.sSortDescending))}else b.setAttribute("aria-label",e+(y[c].asSorting[0]=="asc"?B.sSortAscending:B.sSortDescending));else b.setAttribute("aria-label",e)}a.bSorted=true;i(a.oInstance).trigger("sort",a);if(a.oFeatures.bFilter)X(a,a.oPreviousSearch,1);else{a.aiDisplay=
+a.aiDisplayMaster.slice();a._iDisplayStart=0;I(a);H(a)}}function ya(a,b,c,d){fb(b,{},function(e){if(a.aoColumns[c].bSortable!==false){var f=function(){var g,j;if(e.shiftKey){for(var k=false,m=0;m<a.aaSorting.length;m++)if(a.aaSorting[m][0]==c){k=true;g=a.aaSorting[m][0];j=a.aaSorting[m][2]+1;if(a.aoColumns[g].asSorting[j]){a.aaSorting[m][1]=a.aoColumns[g].asSorting[j];a.aaSorting[m][2]=j}else a.aaSorting.splice(m,1);break}k===false&&a.aaSorting.push([c,a.aoColumns[c].asSorting[0],0])}else if(a.aaSorting.length==
+1&&a.aaSorting[0][0]==c){g=a.aaSorting[0][0];j=a.aaSorting[0][2]+1;a.aoColumns[g].asSorting[j]||(j=0);a.aaSorting[0][1]=a.aoColumns[g].asSorting[j];a.aaSorting[0][2]=j}else{a.aaSorting.splice(0,a.aaSorting.length);a.aaSorting.push([c,a.aoColumns[c].asSorting[0],0])}$(a)};if(a.oFeatures.bProcessing){P(a,true);setTimeout(function(){f();a.oFeatures.bServerSide||P(a,false)},0)}else f();typeof d=="function"&&d(a)}})}function ba(a){var b,c,d,e,f,g=a.aoColumns.length,j=a.oClasses;for(b=0;b<g;b++)a.aoColumns[b].bSortable&&
+i(a.aoColumns[b].nTh).removeClass(j.sSortAsc+" "+j.sSortDesc+" "+a.aoColumns[b].sSortingClass);c=a.aaSortingFixed!==null?a.aaSortingFixed.concat(a.aaSorting):a.aaSorting.slice();for(b=0;b<a.aoColumns.length;b++)if(a.aoColumns[b].bSortable){f=a.aoColumns[b].sSortingClass;e=-1;for(d=0;d<c.length;d++)if(c[d][0]==b){f=c[d][1]=="asc"?j.sSortAsc:j.sSortDesc;e=d;break}i(a.aoColumns[b].nTh).addClass(f);if(a.bJUI){f=i("span."+j.sSortIcon,a.aoColumns[b].nTh);f.removeClass(j.sSortJUIAsc+" "+j.sSortJUIDesc+" "+
+j.sSortJUI+" "+j.sSortJUIAscAllowed+" "+j.sSortJUIDescAllowed);f.addClass(e==-1?a.aoColumns[b].sSortingClassJUI:c[e][1]=="asc"?j.sSortJUIAsc:j.sSortJUIDesc)}}else i(a.aoColumns[b].nTh).addClass(a.aoColumns[b].sSortingClass);f=j.sSortColumn;if(a.oFeatures.bSort&&a.oFeatures.bSortClasses){a=W(a);e=[];for(b=0;b<g;b++)e.push("");b=0;for(d=1;b<c.length;b++){j=parseInt(c[b][0],10);e[j]=f+d;d<3&&d++}f=new RegExp(f+"[123]");var k;b=0;for(c=a.length;b<c;b++){j=b%g;d=a[b].className;k=e[j];j=d.replace(f,k);
+if(j!=d)a[b].className=i.trim(j);else if(k.length>0&&d.indexOf(k)==-1)a[b].className=d+" "+k}}}function Ha(a){if(!(!a.oFeatures.bStateSave||a.bDestroying)){var b,c;b=a.oScroll.bInfinite;var d={iCreate:(new Date).getTime(),iStart:b?0:a._iDisplayStart,iEnd:b?a._iDisplayLength:a._iDisplayEnd,iLength:a._iDisplayLength,aaSorting:i.extend(true,[],a.aaSorting),oSearch:i.extend(true,{},a.oPreviousSearch),aoSearchCols:i.extend(true,[],a.aoPreSearchCols),abVisCols:[]};b=0;for(c=a.aoColumns.length;b<c;b++)d.abVisCols.push(a.aoColumns[b].bVisible);
+K(a,"aoStateSaveParams","stateSaveParams",[a,d]);a.fnStateSave.call(a.oInstance,a,d)}}function gb(a,b){if(a.oFeatures.bStateSave){var c=a.fnStateLoad.call(a.oInstance,a);if(c){var d=K(a,"aoStateLoadParams","stateLoadParams",[a,c]);if(i.inArray(false,d)===-1){a.oLoadedState=i.extend(true,{},c);a._iDisplayStart=c.iStart;a.iInitDisplayStart=c.iStart;a._iDisplayEnd=c.iEnd;a._iDisplayLength=c.iLength;a.aaSorting=c.aaSorting.slice();a.saved_aaSorting=c.aaSorting.slice();i.extend(a.oPreviousSearch,c.oSearch);
+i.extend(true,a.aoPreSearchCols,c.aoSearchCols);b.saved_aoColumns=[];for(d=0;d<c.abVisCols.length;d++){b.saved_aoColumns[d]={};b.saved_aoColumns[d].bVisible=c.abVisCols[d]}K(a,"aoStateLoaded","stateLoaded",[a,c])}}}}function lb(a,b,c,d,e){var f=new Date;f.setTime(f.getTime()+c*1E3);c=la.location.pathname.split("/");a=a+"_"+c.pop().replace(/[\/:]/g,"").toLowerCase();var g;if(e!==null){g=typeof i.parseJSON==="function"?i.parseJSON(b):eval("("+b+")");b=e(a,g,f.toGMTString(),c.join("/")+"/")}else b=a+
+"="+encodeURIComponent(b)+"; expires="+f.toGMTString()+"; path="+c.join("/")+"/";a=s.cookie.split(";");e=b.split(";")[0].length;f=[];if(e+s.cookie.length+10>4096){for(var j=0,k=a.length;j<k;j++)if(a[j].indexOf(d)!=-1){var m=a[j].split("=");try{(g=eval("("+decodeURIComponent(m[1])+")"))&&g.iCreate&&f.push({name:m[0],time:g.iCreate})}catch(u){}}for(f.sort(function(x,y){return y.time-x.time});e+s.cookie.length+10>4096;){if(f.length===0)return;d=f.pop();s.cookie=d.name+"=; expires=Thu, 01-Jan-1970 00:00:01 GMT; path="+
+c.join("/")+"/"}}s.cookie=b}function mb(a){var b=la.location.pathname.split("/");a=a+"_"+b[b.length-1].replace(/[\/:]/g,"").toLowerCase()+"=";b=s.cookie.split(";");for(var c=0;c<b.length;c++){for(var d=b[c];d.charAt(0)==" ";)d=d.substring(1,d.length);if(d.indexOf(a)===0)return decodeURIComponent(d.substring(a.length,d.length))}return null}function C(a){for(var b=0;b<l.settings.length;b++)if(l.settings[b].nTable===a)return l.settings[b];return null}function fa(a){var b=[];a=a.aoData;for(var c=0,d=
+a.length;c<d;c++)a[c].nTr!==null&&b.push(a[c].nTr);return b}function W(a,b){var c=[],d,e,f,g,j;e=0;var k=a.aoData.length;if(b!==p){e=b;k=b+1}for(e=e;e<k;e++){j=a.aoData[e];if(j.nTr!==null){b=[];for(d=j.nTr.firstChild;d;){f=d.nodeName.toLowerCase();if(f=="td"||f=="th")b.push(d);d=d.nextSibling}f=d=0;for(g=a.aoColumns.length;f<g;f++)if(a.aoColumns[f].bVisible)c.push(b[f-d]);else{c.push(j._anHidden[f]);d++}}}return c}function O(a,b,c){a=a===null?"DataTables warning: "+c:"DataTables warning (table id = '"+
+a.sTableId+"'): "+c;if(b===0)if(l.ext.sErrMode=="alert")alert(a);else throw new Error(a);else la.console&&console.log&&console.log(a)}function r(a,b,c,d){if(d===p)d=c;if(b[c]!==p)a[d]=b[c]}function hb(a,b){var c;for(var d in b)if(b.hasOwnProperty(d)){c=b[d];if(typeof h[d]==="object"&&c!==null&&i.isArray(c)===false)i.extend(true,a[d],c);else a[d]=c}return a}function fb(a,b,c){i(a).bind("click.DT",b,function(d){a.blur();c(d)}).bind("keypress.DT",b,function(d){d.which===13&&c(d)}).bind("selectstart.DT",
+function(){return false})}function J(a,b,c,d){c&&a[b].push({fn:c,sName:d})}function K(a,b,c,d){b=a[b];for(var e=[],f=b.length-1;f>=0;f--)e.push(b[f].fn.apply(a.oInstance,d));c!==null&&i(a.oInstance).trigger(c,d);return e}function ib(a){var b=i('<div style="position:absolute; top:0; left:0; height:1px; width:1px; overflow:hidden"><div style="position:absolute; top:1px; left:1px; width:100px; overflow:scroll;"><div id="DT_BrowserTest" style="width:100%; height:10px;"></div></div></div>')[0];s.body.appendChild(b);
+a.oBrowser.bScrollOversize=i("#DT_BrowserTest",b)[0].offsetWidth===100?true:false;s.body.removeChild(b)}function jb(a){return function(){var b=[C(this[l.ext.iApiIndex])].concat(Array.prototype.slice.call(arguments));return l.ext.oApi[a].apply(this,b)}}var ga=/\[.*?\]$/,kb=la.JSON?JSON.stringify:function(a){var b=typeof a;if(b!=="object"||a===null){if(b==="string")a='"'+a+'"';return a+""}var c,d,e=[],f=i.isArray(a);for(c in a){d=a[c];b=typeof d;if(b==="string")d='"'+d+'"';else if(b==="object"&&d!==
+null)d=kb(d);e.push((f?"":'"'+c+'":')+d)}return(f?"[":"{")+e+(f?"]":"}")};this.$=function(a,b){var c,d=[],e;c=C(this[l.ext.iApiIndex]);var f=c.aoData,g=c.aiDisplay,j=c.aiDisplayMaster;b||(b={});b=i.extend({},{filter:"none",order:"current",page:"all"},b);if(b.page=="current"){b=c._iDisplayStart;for(c=c.fnDisplayEnd();b<c;b++)(e=f[g[b]].nTr)&&d.push(e)}else if(b.order=="current"&&b.filter=="none"){b=0;for(c=j.length;b<c;b++)(e=f[j[b]].nTr)&&d.push(e)}else if(b.order=="current"&&b.filter=="applied"){b=
+0;for(c=g.length;b<c;b++)(e=f[g[b]].nTr)&&d.push(e)}else if(b.order=="original"&&b.filter=="none"){b=0;for(c=f.length;b<c;b++)(e=f[b].nTr)&&d.push(e)}else if(b.order=="original"&&b.filter=="applied"){b=0;for(c=f.length;b<c;b++){e=f[b].nTr;i.inArray(b,g)!==-1&&e&&d.push(e)}}else O(c,1,"Unknown selection options");f=i(d);d=f.filter(a);a=f.find(a);return i([].concat(i.makeArray(d),i.makeArray(a)))};this._=function(a,b){var c=[],d=this.$(a,b);a=0;for(b=d.length;a<b;a++)c.push(this.fnGetData(d[a]));return c};
+this.fnAddData=function(a,b){if(a.length===0)return[];var c=[],d,e=C(this[l.ext.iApiIndex]);if(typeof a[0]==="object"&&a[0]!==null)for(var f=0;f<a.length;f++){d=R(e,a[f]);if(d==-1)return c;c.push(d)}else{d=R(e,a);if(d==-1)return c;c.push(d)}e.aiDisplay=e.aiDisplayMaster.slice();if(b===p||b)qa(e);return c};this.fnAdjustColumnSizing=function(a){var b=C(this[l.ext.iApiIndex]);o(b);if(a===p||a)this.fnDraw(false);else if(b.oScroll.sX!==""||b.oScroll.sY!=="")this.oApi._fnScrollDraw(b)};this.fnClearTable=
+function(a){var b=C(this[l.ext.iApiIndex]);wa(b);if(a===p||a)H(b)};this.fnClose=function(a){for(var b=C(this[l.ext.iApiIndex]),c=0;c<b.aoOpenRows.length;c++)if(b.aoOpenRows[c].nParent==a){(a=b.aoOpenRows[c].nTr.parentNode)&&a.removeChild(b.aoOpenRows[c].nTr);b.aoOpenRows.splice(c,1);return 0}return 1};this.fnDeleteRow=function(a,b,c){var d=C(this[l.ext.iApiIndex]),e,f;a=typeof a==="object"?V(d,a):a;var g=d.aoData.splice(a,1);e=0;for(f=d.aoData.length;e<f;e++)if(d.aoData[e].nTr!==null)d.aoData[e].nTr._DT_RowIndex=
+e;e=i.inArray(a,d.aiDisplay);d.asDataSearch.splice(e,1);xa(d.aiDisplayMaster,a);xa(d.aiDisplay,a);typeof b==="function"&&b.call(this,d,g);if(d._iDisplayStart>=d.fnRecordsDisplay()){d._iDisplayStart-=d._iDisplayLength;if(d._iDisplayStart<0)d._iDisplayStart=0}if(c===p||c){I(d);H(d)}return g};this.fnDestroy=function(a){var b=C(this[l.ext.iApiIndex]),c=b.nTableWrapper.parentNode,d=b.nTBody,e,f;a=a===p?false:a;b.bDestroying=true;K(b,"aoDestroyCallback","destroy",[b]);if(!a){e=0;for(f=b.aoColumns.length;e<
+f;e++)b.aoColumns[e].bVisible===false&&this.fnSetColumnVis(e,true)}i(b.nTableWrapper).find("*").andSelf().unbind(".DT");i("tbody>tr>td."+b.oClasses.sRowEmpty,b.nTable).parent().remove();if(b.nTable!=b.nTHead.parentNode){i(b.nTable).children("thead").remove();b.nTable.appendChild(b.nTHead)}if(b.nTFoot&&b.nTable!=b.nTFoot.parentNode){i(b.nTable).children("tfoot").remove();b.nTable.appendChild(b.nTFoot)}b.nTable.parentNode.removeChild(b.nTable);i(b.nTableWrapper).remove();b.aaSorting=[];b.aaSortingFixed=
+[];ba(b);i(fa(b)).removeClass(b.asStripeClasses.join(" "));i("th, td",b.nTHead).removeClass([b.oClasses.sSortable,b.oClasses.sSortableAsc,b.oClasses.sSortableDesc,b.oClasses.sSortableNone].join(" "));if(b.bJUI){i("th span."+b.oClasses.sSortIcon+", td span."+b.oClasses.sSortIcon,b.nTHead).remove();i("th, td",b.nTHead).each(function(){var g=i("div."+b.oClasses.sSortJUIWrapper,this),j=g.contents();i(this).append(j);g.remove()})}if(!a&&b.nTableReinsertBefore)c.insertBefore(b.nTable,b.nTableReinsertBefore);
+else a||c.appendChild(b.nTable);e=0;for(f=b.aoData.length;e<f;e++)b.aoData[e].nTr!==null&&d.appendChild(b.aoData[e].nTr);if(b.oFeatures.bAutoWidth===true)b.nTable.style.width=t(b.sDestroyWidth);if(f=b.asDestroyStripes.length){a=i(d).children("tr");for(e=0;e<f;e++)a.filter(":nth-child("+f+"n + "+e+")").addClass(b.asDestroyStripes[e])}e=0;for(f=l.settings.length;e<f;e++)l.settings[e]==b&&l.settings.splice(e,1);h=b=null};this.fnDraw=function(a){var b=C(this[l.ext.iApiIndex]);if(a===false){I(b);H(b)}else qa(b)};
+this.fnFilter=function(a,b,c,d,e,f){var g=C(this[l.ext.iApiIndex]);if(g.oFeatures.bFilter){if(c===p||c===null)c=false;if(d===p||d===null)d=true;if(e===p||e===null)e=true;if(f===p||f===null)f=true;if(b===p||b===null){X(g,{sSearch:a+"",bRegex:c,bSmart:d,bCaseInsensitive:f},1);if(e&&g.aanFeatures.f){b=g.aanFeatures.f;c=0;for(d=b.length;c<d;c++)try{b[c]._DT_Input!=s.activeElement&&i(b[c]._DT_Input).val(a)}catch(j){i(b[c]._DT_Input).val(a)}}}else{i.extend(g.aoPreSearchCols[b],{sSearch:a+"",bRegex:c,bSmart:d,
+bCaseInsensitive:f});X(g,g.oPreviousSearch,1)}}};this.fnGetData=function(a,b){var c=C(this[l.ext.iApiIndex]);if(a!==p){var d=a;if(typeof a==="object"){var e=a.nodeName.toLowerCase();if(e==="tr")d=V(c,a);else if(e==="td"){d=V(c,a.parentNode);b=va(c,d,a)}}if(b!==p)return F(c,d,b,"");return c.aoData[d]!==p?c.aoData[d]._aData:null}return oa(c)};this.fnGetNodes=function(a){var b=C(this[l.ext.iApiIndex]);if(a!==p)return b.aoData[a]!==p?b.aoData[a].nTr:null;return fa(b)};this.fnGetPosition=function(a){var b=
+C(this[l.ext.iApiIndex]),c=a.nodeName.toUpperCase();if(c=="TR")return V(b,a);else if(c=="TD"||c=="TH"){c=V(b,a.parentNode);a=va(b,c,a);return[c,w(b,a),a]}return null};this.fnIsOpen=function(a){for(var b=C(this[l.ext.iApiIndex]),c=0;c<b.aoOpenRows.length;c++)if(b.aoOpenRows[c].nParent==a)return true;return false};this.fnOpen=function(a,b,c){var d=C(this[l.ext.iApiIndex]),e=fa(d);if(i.inArray(a,e)!==-1){this.fnClose(a);e=s.createElement("tr");var f=s.createElement("td");e.appendChild(f);f.className=
+c;f.colSpan=D(d);if(typeof b==="string")f.innerHTML=b;else i(f).html(b);b=i("tr",d.nTBody);i.inArray(a,b)!=-1&&i(e).insertAfter(a);d.aoOpenRows.push({nTr:e,nParent:a});return e}};this.fnPageChange=function(a,b){var c=C(this[l.ext.iApiIndex]);Ga(c,a);I(c);if(b===p||b)H(c)};this.fnSetColumnVis=function(a,b,c){var d=C(this[l.ext.iApiIndex]),e,f,g=d.aoColumns,j=d.aoData,k,m;if(g[a].bVisible!=b){if(b){for(e=f=0;e<a;e++)g[e].bVisible&&f++;m=f>=D(d);if(!m)for(e=a;e<g.length;e++)if(g[e].bVisible){k=e;break}e=
+0;for(f=j.length;e<f;e++)if(j[e].nTr!==null)m?j[e].nTr.appendChild(j[e]._anHidden[a]):j[e].nTr.insertBefore(j[e]._anHidden[a],W(d,e)[k])}else{e=0;for(f=j.length;e<f;e++)if(j[e].nTr!==null){k=W(d,e)[a];j[e]._anHidden[a]=k;k.parentNode.removeChild(k)}}g[a].bVisible=b;ia(d,d.aoHeader);d.nTFoot&&ia(d,d.aoFooter);e=0;for(f=d.aoOpenRows.length;e<f;e++)d.aoOpenRows[e].nTr.colSpan=D(d);if(c===p||c){o(d);H(d)}Ha(d)}};this.fnSettings=function(){return C(this[l.ext.iApiIndex])};this.fnSort=function(a){var b=
+C(this[l.ext.iApiIndex]);b.aaSorting=a;$(b)};this.fnSortListener=function(a,b,c){ya(C(this[l.ext.iApiIndex]),a,b,c)};this.fnUpdate=function(a,b,c,d,e){var f=C(this[l.ext.iApiIndex]);b=typeof b==="object"?V(f,b):b;if(i.isArray(a)&&c===p){f.aoData[b]._aData=a.slice();for(c=0;c<f.aoColumns.length;c++)this.fnUpdate(F(f,b,c),b,c,false,false)}else if(i.isPlainObject(a)&&c===p){f.aoData[b]._aData=i.extend(true,{},a);for(c=0;c<f.aoColumns.length;c++)this.fnUpdate(F(f,b,c),b,c,false,false)}else{S(f,b,c,a);
+a=F(f,b,c,"display");var g=f.aoColumns[c];if(g.fnRender!==null){a=da(f,b,c);g.bUseRendered&&S(f,b,c,a)}if(f.aoData[b].nTr!==null)W(f,b)[c].innerHTML=a}c=i.inArray(b,f.aiDisplay);f.asDataSearch[c]=Da(f,na(f,b,"filter",A(f,"bSearchable")));if(e===p||e)o(f);if(d===p||d)qa(f);return 0};this.fnVersionCheck=l.ext.fnVersionCheck;this.oApi={_fnExternApiFunc:jb,_fnInitialise:ra,_fnInitComplete:pa,_fnLanguageCompat:Fa,_fnAddColumn:n,_fnColumnOptions:q,_fnAddData:R,_fnCreateTr:ua,_fnGatherData:ea,_fnBuildHead:Ka,
+_fnDrawHead:ia,_fnDraw:H,_fnReDraw:qa,_fnAjaxUpdate:La,_fnAjaxParameters:Ta,_fnAjaxUpdateDraw:Ua,_fnServerParams:Aa,_fnAddOptionsHtml:Ma,_fnFeatureHtmlTable:Qa,_fnScrollDraw:$a,_fnAdjustColumnSizing:o,_fnFeatureHtmlFilter:Oa,_fnFilterComplete:X,_fnFilterCustom:Xa,_fnFilterColumn:Wa,_fnFilter:Va,_fnBuildSearchArray:Ba,_fnBuildSearchRow:Da,_fnFilterCreateSearch:Ca,_fnDataToSearch:Ya,_fnSort:$,_fnSortAttachListener:ya,_fnSortingClasses:ba,_fnFeatureHtmlPaginate:Sa,_fnPageChange:Ga,_fnFeatureHtmlInfo:Ra,
+_fnUpdateInfo:Za,_fnFeatureHtmlLength:Na,_fnFeatureHtmlProcessing:Pa,_fnProcessingDisplay:P,_fnVisibleToColumnIndex:v,_fnColumnIndexToVisible:w,_fnNodeToDataIndex:V,_fnVisbleColumns:D,_fnCalculateEnd:I,_fnConvertToWidth:ab,_fnCalculateColumnWidths:ta,_fnScrollingWidthAdjust:cb,_fnGetWidestNode:bb,_fnGetMaxLenString:db,_fnStringToCss:t,_fnDetectType:G,_fnSettingsFromNode:C,_fnGetDataMaster:oa,_fnGetTrNodes:fa,_fnGetTdNodes:W,_fnEscapeRegex:Ea,_fnDeleteIndex:xa,_fnReOrderIndex:E,_fnColumnOrdering:Y,
+_fnLog:O,_fnClearTable:wa,_fnSaveState:Ha,_fnLoadState:gb,_fnCreateCookie:lb,_fnReadCookie:mb,_fnDetectHeader:ha,_fnGetUniqueThs:Z,_fnScrollBarWidth:eb,_fnApplyToChildren:N,_fnMap:r,_fnGetRowData:na,_fnGetCellData:F,_fnSetCellData:S,_fnGetObjectDataFn:ca,_fnSetObjectDataFn:Ja,_fnApplyColumnDefs:ma,_fnBindAction:fb,_fnExtend:hb,_fnCallbackReg:J,_fnCallbackFire:K,_fnJsonString:kb,_fnRender:da,_fnNodeToColumnIndex:va,_fnInfoMacros:za,_fnBrowserDetect:ib,_fnGetColumns:A};i.extend(l.ext.oApi,this.oApi);
+for(var Ia in l.ext.oApi)if(Ia)this[Ia]=jb(Ia);var sa=this;this.each(function(){var a=0,b,c,d;c=this.getAttribute("id");var e=false,f=false;if(this.nodeName.toLowerCase()!="table")O(null,0,"Attempted to initialise DataTables on a node which is not a table: "+this.nodeName);else{a=0;for(b=l.settings.length;a<b;a++){if(l.settings[a].nTable==this)if(h===p||h.bRetrieve)return l.settings[a].oInstance;else if(h.bDestroy){l.settings[a].oInstance.fnDestroy();break}else{O(l.settings[a],0,"Cannot reinitialise DataTable.\n\nTo retrieve the DataTables object for this table, pass no arguments or see the docs for bRetrieve and bDestroy");
+return}if(l.settings[a].sTableId==this.id){l.settings.splice(a,1);break}}if(c===null||c==="")this.id=c="DataTables_Table_"+l.ext._oExternConfig.iNextUnique++;var g=i.extend(true,{},l.models.oSettings,{nTable:this,oApi:sa.oApi,oInit:h,sDestroyWidth:i(this).width(),sInstance:c,sTableId:c});l.settings.push(g);g.oInstance=sa.length===1?sa:i(this).dataTable();h||(h={});h.oLanguage&&Fa(h.oLanguage);h=hb(i.extend(true,{},l.defaults),h);r(g.oFeatures,h,"bPaginate");r(g.oFeatures,h,"bLengthChange");r(g.oFeatures,
+h,"bFilter");r(g.oFeatures,h,"bSort");r(g.oFeatures,h,"bInfo");r(g.oFeatures,h,"bProcessing");r(g.oFeatures,h,"bAutoWidth");r(g.oFeatures,h,"bSortClasses");r(g.oFeatures,h,"bServerSide");r(g.oFeatures,h,"bDeferRender");r(g.oScroll,h,"sScrollX","sX");r(g.oScroll,h,"sScrollXInner","sXInner");r(g.oScroll,h,"sScrollY","sY");r(g.oScroll,h,"bScrollCollapse","bCollapse");r(g.oScroll,h,"bScrollInfinite","bInfinite");r(g.oScroll,h,"iScrollLoadGap","iLoadGap");r(g.oScroll,h,"bScrollAutoCss","bAutoCss");r(g,
+h,"asStripeClasses");r(g,h,"asStripClasses","asStripeClasses");r(g,h,"fnServerData");r(g,h,"fnFormatNumber");r(g,h,"sServerMethod");r(g,h,"aaSorting");r(g,h,"aaSortingFixed");r(g,h,"aLengthMenu");r(g,h,"sPaginationType");r(g,h,"sAjaxSource");r(g,h,"sAjaxDataProp");r(g,h,"iCookieDuration");r(g,h,"sCookiePrefix");r(g,h,"sDom");r(g,h,"bSortCellsTop");r(g,h,"iTabIndex");r(g,h,"oSearch","oPreviousSearch");r(g,h,"aoSearchCols","aoPreSearchCols");r(g,h,"iDisplayLength","_iDisplayLength");r(g,h,"bJQueryUI",
+"bJUI");r(g,h,"fnCookieCallback");r(g,h,"fnStateLoad");r(g,h,"fnStateSave");r(g.oLanguage,h,"fnInfoCallback");J(g,"aoDrawCallback",h.fnDrawCallback,"user");J(g,"aoServerParams",h.fnServerParams,"user");J(g,"aoStateSaveParams",h.fnStateSaveParams,"user");J(g,"aoStateLoadParams",h.fnStateLoadParams,"user");J(g,"aoStateLoaded",h.fnStateLoaded,"user");J(g,"aoRowCallback",h.fnRowCallback,"user");J(g,"aoRowCreatedCallback",h.fnCreatedRow,"user");J(g,"aoHeaderCallback",h.fnHeaderCallback,"user");J(g,"aoFooterCallback",
+h.fnFooterCallback,"user");J(g,"aoInitComplete",h.fnInitComplete,"user");J(g,"aoPreDrawCallback",h.fnPreDrawCallback,"user");if(g.oFeatures.bServerSide&&g.oFeatures.bSort&&g.oFeatures.bSortClasses)J(g,"aoDrawCallback",ba,"server_side_sort_classes");else g.oFeatures.bDeferRender&&J(g,"aoDrawCallback",ba,"defer_sort_classes");if(h.bJQueryUI){i.extend(g.oClasses,l.ext.oJUIClasses);if(h.sDom===l.defaults.sDom&&l.defaults.sDom==="lfrtip")g.sDom='<"H"lfr>t<"F"ip>'}else i.extend(g.oClasses,l.ext.oStdClasses);
+i(this).addClass(g.oClasses.sTable);if(g.oScroll.sX!==""||g.oScroll.sY!=="")g.oScroll.iBarWidth=eb();if(g.iInitDisplayStart===p){g.iInitDisplayStart=h.iDisplayStart;g._iDisplayStart=h.iDisplayStart}if(h.bStateSave){g.oFeatures.bStateSave=true;gb(g,h);J(g,"aoDrawCallback",Ha,"state_save")}if(h.iDeferLoading!==null){g.bDeferLoading=true;a=i.isArray(h.iDeferLoading);g._iRecordsDisplay=a?h.iDeferLoading[0]:h.iDeferLoading;g._iRecordsTotal=a?h.iDeferLoading[1]:h.iDeferLoading}if(h.aaData!==null)f=true;
+if(h.oLanguage.sUrl!==""){g.oLanguage.sUrl=h.oLanguage.sUrl;i.getJSON(g.oLanguage.sUrl,null,function(k){Fa(k);i.extend(true,g.oLanguage,h.oLanguage,k);ra(g)});e=true}else i.extend(true,g.oLanguage,h.oLanguage);if(h.asStripeClasses===null)g.asStripeClasses=[g.oClasses.sStripeOdd,g.oClasses.sStripeEven];b=g.asStripeClasses.length;g.asDestroyStripes=[];if(b){c=false;d=i(this).children("tbody").children("tr:lt("+b+")");for(a=0;a<b;a++)if(d.hasClass(g.asStripeClasses[a])){c=true;g.asDestroyStripes.push(g.asStripeClasses[a])}c&&
+d.removeClass(g.asStripeClasses.join(" "))}c=[];a=this.getElementsByTagName("thead");if(a.length!==0){ha(g.aoHeader,a[0]);c=Z(g)}if(h.aoColumns===null){d=[];a=0;for(b=c.length;a<b;a++)d.push(null)}else d=h.aoColumns;a=0;for(b=d.length;a<b;a++){if(h.saved_aoColumns!==p&&h.saved_aoColumns.length==b){if(d[a]===null)d[a]={};d[a].bVisible=h.saved_aoColumns[a].bVisible}n(g,c?c[a]:null)}ma(g,h.aoColumnDefs,d,function(k,m){q(g,k,m)});a=0;for(b=g.aaSorting.length;a<b;a++){if(g.aaSorting[a][0]>=g.aoColumns.length)g.aaSorting[a][0]=
+0;var j=g.aoColumns[g.aaSorting[a][0]];if(g.aaSorting[a][2]===p)g.aaSorting[a][2]=0;if(h.aaSorting===p&&g.saved_aaSorting===p)g.aaSorting[a][1]=j.asSorting[0];c=0;for(d=j.asSorting.length;c<d;c++)if(g.aaSorting[a][1]==j.asSorting[c]){g.aaSorting[a][2]=c;break}}ba(g);ib(g);a=i(this).children("caption").each(function(){this._captionSide=i(this).css("caption-side")});b=i(this).children("thead");if(b.length===0){b=[s.createElement("thead")];this.appendChild(b[0])}g.nTHead=b[0];b=i(this).children("tbody");
+if(b.length===0){b=[s.createElement("tbody")];this.appendChild(b[0])}g.nTBody=b[0];g.nTBody.setAttribute("role","alert");g.nTBody.setAttribute("aria-live","polite");g.nTBody.setAttribute("aria-relevant","all");b=i(this).children("tfoot");if(b.length===0&&a.length>0&&(g.oScroll.sX!==""||g.oScroll.sY!=="")){b=[s.createElement("tfoot")];this.appendChild(b[0])}if(b.length>0){g.nTFoot=b[0];ha(g.aoFooter,g.nTFoot)}if(f)for(a=0;a<h.aaData.length;a++)R(g,h.aaData[a]);else ea(g);g.aiDisplay=g.aiDisplayMaster.slice();
+g.bInitialised=true;e===false&&ra(g)}});sa=null;return this};l.fnVersionCheck=function(h){var n=function(A,G){for(;A.length<G;)A+="0";return A},q=l.ext.sVersion.split(".");h=h.split(".");for(var o="",v="",w=0,D=h.length;w<D;w++){o+=n(q[w],3);v+=n(h[w],3)}return parseInt(o,10)>=parseInt(v,10)};l.fnIsDataTable=function(h){for(var n=l.settings,q=0;q<n.length;q++)if(n[q].nTable===h||n[q].nScrollHead===h||n[q].nScrollFoot===h)return true;return false};l.fnTables=function(h){var n=[];jQuery.each(l.settings,
+function(q,o){if(!h||h===true&&i(o.nTable).is(":visible"))n.push(o.nTable)});return n};l.version="1.9.4";l.settings=[];l.models={};l.models.ext={afnFiltering:[],afnSortData:[],aoFeatures:[],aTypes:[],fnVersionCheck:l.fnVersionCheck,iApiIndex:0,ofnSearch:{},oApi:{},oStdClasses:{},oJUIClasses:{},oPagination:{},oSort:{},sVersion:l.version,sErrMode:"alert",_oExternConfig:{iNextUnique:0}};l.models.oSearch={bCaseInsensitive:true,sSearch:"",bRegex:false,bSmart:true};l.models.oRow={nTr:null,_aData:[],_aSortData:[],
+_anHidden:[],_sRowStripe:""};l.models.oColumn={aDataSort:null,asSorting:null,bSearchable:null,bSortable:null,bUseRendered:null,bVisible:null,_bAutoType:true,fnCreatedCell:null,fnGetData:null,fnRender:null,fnSetData:null,mData:null,mRender:null,nTh:null,nTf:null,sClass:null,sContentPadding:null,sDefaultContent:null,sName:null,sSortDataType:"std",sSortingClass:null,sSortingClassJUI:null,sTitle:null,sType:null,sWidth:null,sWidthOrig:null};l.defaults={aaData:null,aaSorting:[[0,"asc"]],aaSortingFixed:null,
+aLengthMenu:[10,25,50,100],aoColumns:null,aoColumnDefs:null,aoSearchCols:[],asStripeClasses:null,bAutoWidth:true,bDeferRender:false,bDestroy:false,bFilter:true,bInfo:true,bJQueryUI:false,bLengthChange:true,bPaginate:true,bProcessing:false,bRetrieve:false,bScrollAutoCss:true,bScrollCollapse:false,bScrollInfinite:false,bServerSide:false,bSort:true,bSortCellsTop:false,bSortClasses:true,bStateSave:false,fnCookieCallback:null,fnCreatedRow:null,fnDrawCallback:null,fnFooterCallback:null,fnFormatNumber:function(h){if(h<
+1E3)return h;var n=h+"";h=n.split("");var q="";n=n.length;for(var o=0;o<n;o++){if(o%3===0&&o!==0)q=this.oLanguage.sInfoThousands+q;q=h[n-o-1]+q}return q},fnHeaderCallback:null,fnInfoCallback:null,fnInitComplete:null,fnPreDrawCallback:null,fnRowCallback:null,fnServerData:function(h,n,q,o){o.jqXHR=i.ajax({url:h,data:n,success:function(v){v.sError&&o.oApi._fnLog(o,0,v.sError);i(o.oInstance).trigger("xhr",[o,v]);q(v)},dataType:"json",cache:false,type:o.sServerMethod,error:function(v,w){w=="parsererror"&&
+o.oApi._fnLog(o,0,"DataTables warning: JSON data from server could not be parsed. This is caused by a JSON formatting error.")}})},fnServerParams:null,fnStateLoad:function(h){h=this.oApi._fnReadCookie(h.sCookiePrefix+h.sInstance);var n;try{n=typeof i.parseJSON==="function"?i.parseJSON(h):eval("("+h+")")}catch(q){n=null}return n},fnStateLoadParams:null,fnStateLoaded:null,fnStateSave:function(h,n){this.oApi._fnCreateCookie(h.sCookiePrefix+h.sInstance,this.oApi._fnJsonString(n),h.iCookieDuration,h.sCookiePrefix,
+h.fnCookieCallback)},fnStateSaveParams:null,iCookieDuration:7200,iDeferLoading:null,iDisplayLength:10,iDisplayStart:0,iScrollLoadGap:100,iTabIndex:0,oLanguage:{oAria:{sSortAscending:": activate to sort column ascending",sSortDescending:": activate to sort column descending"},oPaginate:{sFirst:"First",sLast:"Last",sNext:"Next",sPrevious:"Previous"},sEmptyTable:"No data available in table",sInfo:"Showing _START_ to _END_ of _TOTAL_ entries",sInfoEmpty:"Showing 0 to 0 of 0 entries",sInfoFiltered:"(filtered from _MAX_ total entries)",
+sInfoPostFix:"",sInfoThousands:",",sLengthMenu:"Show _MENU_ entries",sLoadingRecords:"Loading...",sProcessing:"Processing...",sSearch:"Search:",sUrl:"",sZeroRecords:"No matching records found"},oSearch:i.extend({},l.models.oSearch),sAjaxDataProp:"aaData",sAjaxSource:null,sCookiePrefix:"SpryMedia_DataTables_",sDom:"lfrtip",sPaginationType:"two_button",sScrollX:"",sScrollXInner:"",sScrollY:"",sServerMethod:"GET"};l.defaults.columns={aDataSort:null,asSorting:["asc","desc"],bSearchable:true,bSortable:true,
+bUseRendered:true,bVisible:true,fnCreatedCell:null,fnRender:null,iDataSort:-1,mData:null,mRender:null,sCellType:"td",sClass:"",sContentPadding:"",sDefaultContent:null,sName:"",sSortDataType:"std",sTitle:null,sType:null,sWidth:null};l.models.oSettings={oFeatures:{bAutoWidth:null,bDeferRender:null,bFilter:null,bInfo:null,bLengthChange:null,bPaginate:null,bProcessing:null,bServerSide:null,bSort:null,bSortClasses:null,bStateSave:null},oScroll:{bAutoCss:null,bCollapse:null,bInfinite:null,iBarWidth:0,iLoadGap:null,
+sX:null,sXInner:null,sY:null},oLanguage:{fnInfoCallback:null},oBrowser:{bScrollOversize:false},aanFeatures:[],aoData:[],aiDisplay:[],aiDisplayMaster:[],aoColumns:[],aoHeader:[],aoFooter:[],asDataSearch:[],oPreviousSearch:{},aoPreSearchCols:[],aaSorting:null,aaSortingFixed:null,asStripeClasses:null,asDestroyStripes:[],sDestroyWidth:0,aoRowCallback:[],aoHeaderCallback:[],aoFooterCallback:[],aoDrawCallback:[],aoRowCreatedCallback:[],aoPreDrawCallback:[],aoInitComplete:[],aoStateSaveParams:[],aoStateLoadParams:[],
+aoStateLoaded:[],sTableId:"",nTable:null,nTHead:null,nTFoot:null,nTBody:null,nTableWrapper:null,bDeferLoading:false,bInitialised:false,aoOpenRows:[],sDom:null,sPaginationType:"two_button",iCookieDuration:0,sCookiePrefix:"",fnCookieCallback:null,aoStateSave:[],aoStateLoad:[],oLoadedState:null,sAjaxSource:null,sAjaxDataProp:null,bAjaxDataGet:true,jqXHR:null,fnServerData:null,aoServerParams:[],sServerMethod:null,fnFormatNumber:null,aLengthMenu:null,iDraw:0,bDrawing:false,iDrawError:-1,_iDisplayLength:10,
+_iDisplayStart:0,_iDisplayEnd:10,_iRecordsTotal:0,_iRecordsDisplay:0,bJUI:null,oClasses:{},bFiltered:false,bSorted:false,bSortCellsTop:null,oInit:null,aoDestroyCallback:[],fnRecordsTotal:function(){return this.oFeatures.bServerSide?parseInt(this._iRecordsTotal,10):this.aiDisplayMaster.length},fnRecordsDisplay:function(){return this.oFeatures.bServerSide?parseInt(this._iRecordsDisplay,10):this.aiDisplay.length},fnDisplayEnd:function(){return this.oFeatures.bServerSide?this.oFeatures.bPaginate===false||
+this._iDisplayLength==-1?this._iDisplayStart+this.aiDisplay.length:Math.min(this._iDisplayStart+this._iDisplayLength,this._iRecordsDisplay):this._iDisplayEnd},oInstance:null,sInstance:null,iTabIndex:0,nScrollHead:null,nScrollFoot:null};l.ext=i.extend(true,{},l.models.ext);i.extend(l.ext.oStdClasses,{sTable:"dataTable",sPagePrevEnabled:"paginate_enabled_previous",sPagePrevDisabled:"paginate_disabled_previous",sPageNextEnabled:"paginate_enabled_next",sPageNextDisabled:"paginate_disabled_next",sPageJUINext:"",
+sPageJUIPrev:"",sPageButton:"paginate_button",sPageButtonActive:"paginate_active",sPageButtonStaticDisabled:"paginate_button paginate_button_disabled",sPageFirst:"first",sPagePrevious:"previous",sPageNext:"next",sPageLast:"last",sStripeOdd:"odd",sStripeEven:"even",sRowEmpty:"dataTables_empty",sWrapper:"dataTables_wrapper",sFilter:"dataTables_filter",sInfo:"dataTables_info",sPaging:"dataTables_paginate paging_",sLength:"dataTables_length",sProcessing:"dataTables_processing",sSortAsc:"sorting_asc",
+sSortDesc:"sorting_desc",sSortable:"sorting",sSortableAsc:"sorting_asc_disabled",sSortableDesc:"sorting_desc_disabled",sSortableNone:"sorting_disabled",sSortColumn:"sorting_",sSortJUIAsc:"",sSortJUIDesc:"",sSortJUI:"",sSortJUIAscAllowed:"",sSortJUIDescAllowed:"",sSortJUIWrapper:"",sSortIcon:"",sScrollWrapper:"dataTables_scroll",sScrollHead:"dataTables_scrollHead",sScrollHeadInner:"dataTables_scrollHeadInner",sScrollBody:"dataTables_scrollBody",sScrollFoot:"dataTables_scrollFoot",sScrollFootInner:"dataTables_scrollFootInner",
+sFooterTH:"",sJUIHeader:"",sJUIFooter:""});i.extend(l.ext.oJUIClasses,l.ext.oStdClasses,{sPagePrevEnabled:"fg-button ui-button ui-state-default ui-corner-left",sPagePrevDisabled:"fg-button ui-button ui-state-default ui-corner-left ui-state-disabled",sPageNextEnabled:"fg-button ui-button ui-state-default ui-corner-right",sPageNextDisabled:"fg-button ui-button ui-state-default ui-corner-right ui-state-disabled",sPageJUINext:"ui-icon ui-icon-circle-arrow-e",sPageJUIPrev:"ui-icon ui-icon-circle-arrow-w",
+sPageButton:"fg-button ui-button ui-state-default",sPageButtonActive:"fg-button ui-button ui-state-default ui-state-disabled",sPageButtonStaticDisabled:"fg-button ui-button ui-state-default ui-state-disabled",sPageFirst:"first ui-corner-tl ui-corner-bl",sPageLast:"last ui-corner-tr ui-corner-br",sPaging:"dataTables_paginate fg-buttonset ui-buttonset fg-buttonset-multi ui-buttonset-multi paging_",sSortAsc:"ui-state-default",sSortDesc:"ui-state-default",sSortable:"ui-state-default",sSortableAsc:"ui-state-default",
+sSortableDesc:"ui-state-default",sSortableNone:"ui-state-default",sSortJUIAsc:"css_right ui-icon ui-icon-triangle-1-n",sSortJUIDesc:"css_right ui-icon ui-icon-triangle-1-s",sSortJUI:"css_right ui-icon ui-icon-carat-2-n-s",sSortJUIAscAllowed:"css_right ui-icon ui-icon-carat-1-n",sSortJUIDescAllowed:"css_right ui-icon ui-icon-carat-1-s",sSortJUIWrapper:"DataTables_sort_wrapper",sSortIcon:"DataTables_sort_icon",sScrollHead:"dataTables_scrollHead ui-state-default",sScrollFoot:"dataTables_scrollFoot ui-state-default",
+sFooterTH:"ui-state-default",sJUIHeader:"fg-toolbar ui-toolbar ui-widget-header ui-corner-tl ui-corner-tr ui-helper-clearfix",sJUIFooter:"fg-toolbar ui-toolbar ui-widget-header ui-corner-bl ui-corner-br ui-helper-clearfix"});i.extend(l.ext.oPagination,{two_button:{fnInit:function(h,n,q){var o=h.oLanguage.oPaginate,v=function(D){h.oApi._fnPageChange(h,D.data.action)&&q(h)};o=!h.bJUI?'<a class="'+h.oClasses.sPagePrevDisabled+'" tabindex="'+h.iTabIndex+'" role="button">'+o.sPrevious+'</a><a class="'+
+h.oClasses.sPageNextDisabled+'" tabindex="'+h.iTabIndex+'" role="button">'+o.sNext+"</a>":'<a class="'+h.oClasses.sPagePrevDisabled+'" tabindex="'+h.iTabIndex+'" role="button"><span class="'+h.oClasses.sPageJUIPrev+'"></span></a><a class="'+h.oClasses.sPageNextDisabled+'" tabindex="'+h.iTabIndex+'" role="button"><span class="'+h.oClasses.sPageJUINext+'"></span></a>';i(n).append(o);var w=i("a",n);o=w[0];w=w[1];h.oApi._fnBindAction(o,{action:"previous"},v);h.oApi._fnBindAction(w,{action:"next"},v);
+if(!h.aanFeatures.p){n.id=h.sTableId+"_paginate";o.id=h.sTableId+"_previous";w.id=h.sTableId+"_next";o.setAttribute("aria-controls",h.sTableId);w.setAttribute("aria-controls",h.sTableId)}},fnUpdate:function(h){if(h.aanFeatures.p)for(var n=h.oClasses,q=h.aanFeatures.p,o,v=0,w=q.length;v<w;v++)if(o=q[v].firstChild){o.className=h._iDisplayStart===0?n.sPagePrevDisabled:n.sPagePrevEnabled;o=o.nextSibling;o.className=h.fnDisplayEnd()==h.fnRecordsDisplay()?n.sPageNextDisabled:n.sPageNextEnabled}}},iFullNumbersShowPages:5,
+full_numbers:{fnInit:function(h,n,q){var o=h.oLanguage.oPaginate,v=h.oClasses,w=function(G){h.oApi._fnPageChange(h,G.data.action)&&q(h)};i(n).append('<a  tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPageFirst+'">'+o.sFirst+'</a><a  tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPagePrevious+'">'+o.sPrevious+'</a><span></span><a tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPageNext+'">'+o.sNext+'</a><a tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPageLast+
+'">'+o.sLast+"</a>");var D=i("a",n);o=D[0];v=D[1];var A=D[2];D=D[3];h.oApi._fnBindAction(o,{action:"first"},w);h.oApi._fnBindAction(v,{action:"previous"},w);h.oApi._fnBindAction(A,{action:"next"},w);h.oApi._fnBindAction(D,{action:"last"},w);if(!h.aanFeatures.p){n.id=h.sTableId+"_paginate";o.id=h.sTableId+"_first";v.id=h.sTableId+"_previous";A.id=h.sTableId+"_next";D.id=h.sTableId+"_last"}},fnUpdate:function(h,n){if(h.aanFeatures.p){var q=l.ext.oPagination.iFullNumbersShowPages,o=Math.floor(q/2),v=
+Math.ceil(h.fnRecordsDisplay()/h._iDisplayLength),w=Math.ceil(h._iDisplayStart/h._iDisplayLength)+1,D="",A,G=h.oClasses,E,Y=h.aanFeatures.p,ma=function(R){h.oApi._fnBindAction(this,{page:R+A-1},function(ea){h.oApi._fnPageChange(h,ea.data.page);n(h);ea.preventDefault()})};if(h._iDisplayLength===-1)w=o=A=1;else if(v<q){A=1;o=v}else if(w<=o){A=1;o=q}else if(w>=v-o){A=v-q+1;o=v}else{A=w-Math.ceil(q/2)+1;o=A+q-1}for(q=A;q<=o;q++)D+=w!==q?'<a tabindex="'+h.iTabIndex+'" class="'+G.sPageButton+'">'+h.fnFormatNumber(q)+
+"</a>":'<a tabindex="'+h.iTabIndex+'" class="'+G.sPageButtonActive+'">'+h.fnFormatNumber(q)+"</a>";q=0;for(o=Y.length;q<o;q++){E=Y[q];if(E.hasChildNodes()){i("span:eq(0)",E).html(D).children("a").each(ma);E=E.getElementsByTagName("a");E=[E[0],E[1],E[E.length-2],E[E.length-1]];i(E).removeClass(G.sPageButton+" "+G.sPageButtonActive+" "+G.sPageButtonStaticDisabled);i([E[0],E[1]]).addClass(w==1?G.sPageButtonStaticDisabled:G.sPageButton);i([E[2],E[3]]).addClass(v===0||w===v||h._iDisplayLength===-1?G.sPageButtonStaticDisabled:
+G.sPageButton)}}}}}});i.extend(l.ext.oSort,{"string-pre":function(h){if(typeof h!="string")h=h!==null&&h.toString?h.toString():"";return h.toLowerCase()},"string-asc":function(h,n){return h<n?-1:h>n?1:0},"string-desc":function(h,n){return h<n?1:h>n?-1:0},"html-pre":function(h){return h.replace(/<.*?>/g,"").toLowerCase()},"html-asc":function(h,n){return h<n?-1:h>n?1:0},"html-desc":function(h,n){return h<n?1:h>n?-1:0},"date-pre":function(h){h=Date.parse(h);if(isNaN(h)||h==="")h=Date.parse("01/01/1970 00:00:00");
+return h},"date-asc":function(h,n){return h-n},"date-desc":function(h,n){return n-h},"numeric-pre":function(h){return h=="-"||h===""?0:h*1},"numeric-asc":function(h,n){return h-n},"numeric-desc":function(h,n){return n-h}});i.extend(l.ext.aTypes,[function(h){if(typeof h==="number")return"numeric";else if(typeof h!=="string")return null;var n,q=false;n=h.charAt(0);if("0123456789-".indexOf(n)==-1)return null;for(var o=1;o<h.length;o++){n=h.charAt(o);if("0123456789.".indexOf(n)==-1)return null;if(n==
+"."){if(q)return null;q=true}}return"numeric"},function(h){var n=Date.parse(h);if(n!==null&&!isNaN(n)||typeof h==="string"&&h.length===0)return"date";return null},function(h){if(typeof h==="string"&&h.indexOf("<")!=-1&&h.indexOf(">")!=-1)return"html";return null}]);i.fn.DataTable=l;i.fn.dataTable=l;i.fn.dataTableSettings=l.settings;i.fn.dataTableExt=l.ext})})(window,document);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js.gz
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js.gz b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js.gz
deleted file mode 100644
index f18b550..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js.gz and /dev/null differ


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


[39/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
HADOOP-10075. Update jetty dependency to version 9 (rkanter)


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

Branch: refs/heads/HDFS-9806
Commit: 5877f20f9c3f6f0afa505715e9a2ee312475af17
Parents: 9e03ee5
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Oct 27 16:01:23 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Oct 27 16:09:00 2016 -0700

----------------------------------------------------------------------
 hadoop-client/pom.xml                           |   20 +-
 .../hadoop-auth-examples/pom.xml                |    2 +-
 .../examples/RequestLoggerFilter.java           |   12 +
 hadoop-common-project/hadoop-auth/pom.xml       |   13 +-
 .../client/AuthenticatorTestCase.java           |   29 +-
 hadoop-common-project/hadoop-common/pom.xml     |   34 +-
 .../hadoop/http/AdminAuthorizedServlet.java     |    2 +-
 .../org/apache/hadoop/http/HttpRequestLog.java  |    4 +-
 .../org/apache/hadoop/http/HttpServer2.java     |  305 +-
 .../java/org/apache/hadoop/http/JettyUtils.java |   35 +
 .../ssl/SslSelectChannelConnectorSecure.java    |   58 -
 .../org/apache/hadoop/conf/TestConfServlet.java |    2 +-
 .../hadoop/fs/FSMainOperationsBaseTest.java     |    4 +-
 .../fs/viewfs/ViewFileSystemTestSetup.java      |   10 +-
 .../hadoop/fs/viewfs/ViewFsTestSetup.java       |   10 +-
 .../http/TestAuthenticationSessionCookie.java   |   11 +-
 .../apache/hadoop/http/TestHttpRequestLog.java  |    4 +-
 .../org/apache/hadoop/http/TestHttpServer.java  |   22 +-
 .../apache/hadoop/http/TestServletFilter.java   |    7 +-
 .../hadoop/http/resource/JerseyResource.java    |    5 +-
 .../delegation/web/TestWebDelegationToken.java  |   64 +-
 hadoop-common-project/hadoop-kms/pom.xml        |   20 +-
 .../hadoop/crypto/key/kms/server/KMS.java       |   21 +-
 .../key/kms/server/KMSAuthenticationFilter.java |   12 +
 .../crypto/key/kms/server/KMSJSONWriter.java    |    3 +-
 .../hadoop/crypto/key/kms/server/MiniKMS.java   |   63 +-
 hadoop-common-project/hadoop-nfs/pom.xml        |    2 +-
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml  |   26 +-
 .../hadoop/fs/http/server/HttpFSServer.java     |   12 +-
 .../apache/hadoop/lib/wsrs/JSONMapProvider.java |    3 +-
 .../apache/hadoop/lib/wsrs/JSONProvider.java    |    3 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |    6 +-
 .../hadoop/fs/http/server/TestHttpFSServer.java |    6 +-
 .../fs/http/server/TestHttpFSServerNoACLs.java  |    6 +-
 .../http/server/TestHttpFSServerNoXAttrs.java   |    6 +-
 .../fs/http/server/TestHttpFSWithKerberos.java  |    6 +-
 .../org/apache/hadoop/test/TestHFSTestCase.java |    8 +-
 .../org/apache/hadoop/test/TestHTestCase.java   |    8 +-
 .../org/apache/hadoop/test/TestJettyHelper.java |   56 +-
 hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml     |    8 +-
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   25 +-
 .../hdfs/qjournal/server/JournalNode.java       |    2 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |    2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |    2 +-
 .../hadoop/hdfs/server/namenode/NNStorage.java  |    2 +-
 .../hdfs/server/namenode/TransferFsImage.java   |    2 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |   39 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |    2 +-
 .../qjournal/server/TestJournalNodeMXBean.java  |    2 +-
 .../blockmanagement/TestBlockStatsMXBean.java   |    2 +-
 .../server/datanode/TestDataNodeMXBean.java     |    2 +-
 .../server/namenode/TestFSNamesystemMBean.java  |    2 +-
 .../server/namenode/TestNameNodeMXBean.java     |    2 +-
 .../namenode/TestStartupProgressServlet.java    |    2 +-
 .../server/namenode/TestTransferFsImage.java    |    2 +-
 .../hadoop/hdfs/web/TestWebHDFSForHA.java       |    2 +-
 .../hadoop/test/MiniDFSClusterManager.java      |    2 +-
 .../hadoop/mapreduce/v2/app/JobEndNotifier.java |   37 +-
 .../mapreduce/v2/app/webapp/AMWebServices.java  |   49 +-
 .../v2/app/webapp/TestAMWebServices.java        |   31 +-
 .../v2/app/webapp/TestAMWebServicesAttempt.java |   13 +-
 .../app/webapp/TestAMWebServicesAttempts.java   |   34 +-
 .../v2/app/webapp/TestAMWebServicesJobConf.java |   13 +-
 .../v2/app/webapp/TestAMWebServicesJobs.java    |   64 +-
 .../v2/app/webapp/TestAMWebServicesTasks.java   |   61 +-
 .../mapreduce/v2/hs/webapp/HsWebServices.java   |   40 +-
 .../v2/hs/webapp/TestHsWebServices.java         |   25 +-
 .../v2/hs/webapp/TestHsWebServicesAttempts.java |   34 +-
 .../v2/hs/webapp/TestHsWebServicesJobConf.java  |   13 +-
 .../v2/hs/webapp/TestHsWebServicesJobs.java     |   67 +-
 .../hs/webapp/TestHsWebServicesJobsQuery.java   |   76 +-
 .../v2/hs/webapp/TestHsWebServicesTasks.java    |   61 +-
 .../hadoop/mapred/NotificationTestCase.java     |   12 +-
 .../mapreduce/MiniHadoopClusterManager.java     |    2 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    |   14 +-
 .../hadoop/mapred/TestShuffleHandler.java       |   25 +-
 .../hadoop-mapreduce-client/pom.xml             |    4 +-
 hadoop-mapreduce-project/pom.xml                |    4 +-
 hadoop-maven-plugins/pom.xml                    |    4 +
 .../maven/plugin/resourcegz/ResourceGzMojo.java |  125 +
 hadoop-project/pom.xml                          |   52 +-
 hadoop-tools/hadoop-azure-datalake/pom.xml      |    2 +-
 hadoop-tools/hadoop-azure/pom.xml               |    7 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |    2 +-
 .../hadoop/fs/azure/BlockBlobAppendStream.java  |    6 +-
 hadoop-tools/hadoop-sls/pom.xml                 |   10 +-
 .../apache/hadoop/yarn/sls/web/SLSWebApp.java   |   19 +-
 .../yarn/client/api/impl/TestAMRMClient.java    |    8 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   10 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |   17 +-
 .../static/dt-1.9.4/js/jquery.dataTables.min.js |  157 +
 .../dt-1.9.4/js/jquery.dataTables.min.js.gz     |  Bin 21726 -> 0 bytes
 .../webapps/static/jquery/jquery-1.8.2.min.js   |    2 +
 .../static/jquery/jquery-1.8.2.min.js.gz        |  Bin 33430 -> 0 bytes
 .../static/jquery/jquery-ui-1.9.1.custom.min.js |    6 +
 .../jquery/jquery-ui-1.9.1.custom.min.js.gz     |  Bin 62855 -> 0 bytes
 .../webapps/static/jt/jquery.jstree.js          | 4544 ++++++++++++++++++
 .../webapps/static/jt/jquery.jstree.js.gz       |  Bin 37540 -> 0 bytes
 .../hadoop/yarn/webapp/MyTestWebService.java    |    3 +-
 .../pom.xml                                     |    2 +-
 .../ApplicationHistoryServer.java               |   50 +-
 .../webapp/AHSWebServices.java                  |   27 +-
 .../timeline/webapp/TimelineWebServices.java    |   23 +-
 .../webapp/TestAHSWebServices.java              |   22 +-
 .../TestRollingLevelDBTimelineStore.java        |    6 +-
 .../webapp/TestTimelineWebServices.java         |  141 +-
 .../hadoop-yarn-server-nodemanager/pom.xml      |    4 +-
 .../nodemanager/webapp/NMWebServices.java       |   28 +-
 .../amrmproxy/MockResourceManagerFacade.java    |   11 +-
 .../TestLogAggregationService.java              |    2 +-
 .../nodemanager/webapp/TestNMWebServices.java   |   27 +-
 .../webapp/TestNMWebServicesApps.java           |   49 +-
 .../webapp/TestNMWebServicesContainers.java     |   25 +-
 .../hadoop-yarn-server-resourcemanager/pom.xml  |    8 +-
 .../yarn/server/resourcemanager/RMNMInfo.java   |    2 +-
 .../resourcemanager/webapp/RMWebServices.java   |  121 +-
 .../yarn/server/resourcemanager/MockNM.java     |    4 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |    4 +-
 .../planning/TestAlignedPlanner.java            |    4 +-
 .../planning/TestGreedyReservationAgent.java    |    4 +-
 .../webapp/TestRMWebServices.java               |   46 +-
 .../webapp/TestRMWebServicesApps.java           |  151 +-
 .../TestRMWebServicesAppsModification.java      |   40 +-
 .../webapp/TestRMWebServicesCapacitySched.java  |   19 +-
 .../TestRMWebServicesDelegationTokens.java      |    4 +-
 .../webapp/TestRMWebServicesFairScheduler.java  |   10 +-
 .../TestRMWebServicesForCSWithPartitions.java   |   13 +-
 .../webapp/TestRMWebServicesNodeLabels.java     |   55 +-
 .../webapp/TestRMWebServicesNodes.java          |   58 +-
 .../webapp/TestRMWebServicesReservation.java    |   13 +-
 .../TestRMWebServicesSchedulerActivities.java   |   91 +-
 .../yarn/webapp/TestRMWithCSRFFilter.java       |    7 +-
 .../pom.xml                                     |    2 +-
 .../pom.xml                                     |   16 +
 ...stTimelineReaderWebServicesHBaseStorage.java |   50 +-
 .../hadoop-yarn-server-timelineservice/pom.xml  |   18 +-
 .../collector/TimelineCollectorWebService.java  |    4 +-
 .../reader/TimelineReaderServer.java            |   25 +-
 .../reader/TimelineReaderWebServices.java       |   47 +-
 .../reader/TestTimelineReaderWebServices.java   |   73 +-
 .../hadoop-yarn-server-web-proxy/pom.xml        |    6 +-
 .../server/webproxy/TestWebAppProxyServlet.java |   39 +-
 pom.xml                                         |    3 +-
 143 files changed, 6868 insertions(+), 1161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client/pom.xml b/hadoop-client/pom.xml
index b330a3d..988dd03 100644
--- a/hadoop-client/pom.xml
+++ b/hadoop-client/pom.xml
@@ -41,7 +41,7 @@
       <exclusions>
         <exclusion>
           <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
         <exclusion>
           <groupId>commons-logging</groupId>
@@ -49,18 +49,18 @@
         </exclusion>
         <exclusion>
           <groupId>jetty</groupId>
-          <artifactId>org.mortbay.jetty</artifactId>
+          <artifactId>org.eclipse.jetty</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>jetty-util</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>servlet-api-2.5</artifactId>
         </exclusion>
         <exclusion>
@@ -112,8 +112,8 @@
           <artifactId>avro</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
           <groupId>com.sun.jersey</groupId>
@@ -125,7 +125,7 @@
         </exclusion>
         <exclusion>
           <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
       </exclusions>
     </dependency>
@@ -137,7 +137,7 @@
       <exclusions>
         <exclusion>
           <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
         <exclusion>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-auth-examples/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth-examples/pom.xml b/hadoop-common-project/hadoop-auth-examples/pom.xml
index c36c157..d841ace 100644
--- a/hadoop-common-project/hadoop-auth-examples/pom.xml
+++ b/hadoop-common-project/hadoop-auth-examples/pom.xml
@@ -34,7 +34,7 @@
   <dependencies>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java
index a9721c9..8048991 100644
--- a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java
+++ b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java
@@ -139,7 +139,19 @@ public class RequestLoggerFilter implements Filter {
       status = sc;
     }
 
+    /**
+     * Calls setStatus(int sc, String msg) on the wrapped
+     * {@link HttpServletResponseWrapper} object.
+     *
+     * @param sc the status code
+     * @param msg the status message
+     * @deprecated {@link HttpServletResponseWrapper#setStatus(int, String)} is
+     * deprecated. To set a status code use {@link #setStatus(int)}, to send an
+     * error with a description use {@link #sendError(int, String)}
+     */
     @Override
+    @Deprecated
+    @SuppressWarnings("deprecation")
     public void setStatus(int sc, String msg) {
       super.setStatus(sc, msg);
       status = sc;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-auth/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml
index 0b37715..5df4bdb 100644
--- a/hadoop-common-project/hadoop-auth/pom.xml
+++ b/hadoop-common-project/hadoop-auth/pom.xml
@@ -53,13 +53,18 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
       <scope>test</scope>
     </dependency>
      <dependency>
@@ -74,7 +79,7 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
index 35e40d8..8b9d45e 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
@@ -30,11 +30,14 @@ import org.apache.http.impl.auth.SPNegoScheme;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.util.EntityUtils;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.FilterHolder;
-import org.mortbay.jetty.servlet.ServletHolder;
-
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import javax.servlet.DispatcherType;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
@@ -53,6 +56,7 @@ import java.net.HttpURLConnection;
 import java.net.ServerSocket;
 import java.net.URL;
 import java.security.Principal;
+import java.util.EnumSet;
 import java.util.Properties;
 
 import org.junit.Assert;
@@ -63,7 +67,7 @@ public class AuthenticatorTestCase {
   private int port = -1;
   private boolean useTomcat = false;
   private Tomcat tomcat = null;
-  Context context;
+  ServletContextHandler context;
 
   private static Properties authenticatorConfig;
 
@@ -121,16 +125,19 @@ public class AuthenticatorTestCase {
   }
 
   protected void startJetty() throws Exception {
-    server = new Server(0);
-    context = new Context();
+    server = new Server();
+    context = new ServletContextHandler();
     context.setContextPath("/foo");
     server.setHandler(context);
-    context.addFilter(new FilterHolder(TestFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(TestFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(TestServlet.class), "/bar");
     host = "localhost";
     port = getLocalPort();
-    server.getConnectors()[0].setHost(host);
-    server.getConnectors()[0].setPort(port);
+    ServerConnector connector = new ServerConnector(server);
+    connector.setHost(host);
+    connector.setPort(port);
+    server.setConnectors(new Connector[] {connector});
     server.start();
     System.out.println("Running embedded servlet container at: http://" + host + ":" + port);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 fd9b7cd..645d495 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -93,25 +93,35 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-sslengine</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-webapp</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>javax.servlet.jsp</groupId>
       <artifactId>jsp-api</artifactId>
       <scope>runtime</scope>
@@ -412,6 +422,18 @@
               <output>${project.build.directory}/generated-test-sources/java</output>
             </configuration>
           </execution>
+          <execution>
+            <id>resource-gz</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>resource-gz</goal>
+            </goals>
+            <configuration>
+              <inputDirectory>${basedir}/src/main/webapps/static</inputDirectory>
+              <outputDirectory>${basedir}/target/webapps/static</outputDirectory>
+              <extensions>js,css</extensions>
+            </configuration>
+          </execution>
         </executions>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
index ef562b4..a4b05a1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
@@ -23,7 +23,7 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.mortbay.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.DefaultServlet;
 
 /**
  * General servlet which is admin-authorized.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java
index 52d9850..a7c23b9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java
@@ -25,8 +25,8 @@ import org.apache.commons.logging.LogConfigurationException;
 import org.apache.commons.logging.LogFactory;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Logger;
-import org.mortbay.jetty.NCSARequestLog;
-import org.mortbay.jetty.RequestLog;
+import org.eclipse.jetty.server.NCSARequestLog;
+import org.eclipse.jetty.server.RequestLog;
 
 /**
  * RequestLog object for use with Http

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index a2bb18f..49ec90a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -56,7 +56,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.AuthenticationFilterInitializer;
 import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
-import org.apache.hadoop.security.ssl.SslSelectChannelConnectorSecure;
 import org.apache.hadoop.jmx.JMXJsonServlet;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.security.SecurityUtil;
@@ -65,34 +64,39 @@ import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Shell;
-import org.mortbay.io.Buffer;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Handler;
-import org.mortbay.jetty.MimeTypes;
-import org.mortbay.jetty.RequestLog;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.SessionManager;
-import org.mortbay.jetty.handler.ContextHandler;
-import org.mortbay.jetty.handler.ContextHandlerCollection;
-import org.mortbay.jetty.handler.HandlerCollection;
-import org.mortbay.jetty.handler.RequestLogHandler;
-import org.mortbay.jetty.nio.SelectChannelConnector;
-import org.mortbay.jetty.security.SslSelectChannelConnector;
-import org.mortbay.jetty.servlet.AbstractSessionManager;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.DefaultServlet;
-import org.mortbay.jetty.servlet.FilterHolder;
-import org.mortbay.jetty.servlet.SessionHandler;
-import org.mortbay.jetty.servlet.FilterMapping;
-import org.mortbay.jetty.servlet.ServletHandler;
-import org.mortbay.jetty.servlet.ServletHolder;
-import org.mortbay.jetty.webapp.WebAppContext;
-import org.mortbay.thread.QueuedThreadPool;
-import org.mortbay.util.MultiException;
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.ConnectionFactory;
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SessionManager;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.server.handler.ContextHandlerCollection;
+import org.eclipse.jetty.server.handler.HandlerCollection;
+import org.eclipse.jetty.server.handler.RequestLogHandler;
+import org.eclipse.jetty.server.session.AbstractSessionManager;
+import org.eclipse.jetty.server.session.SessionHandler;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.FilterMapping;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.ServletMapping;
+import org.eclipse.jetty.util.ArrayUtil;
+import org.eclipse.jetty.util.MultiException;
+import org.eclipse.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.sun.jersey.spi.container.servlet.ServletContainer;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
 
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal is
@@ -126,11 +130,13 @@ public final class HttpServer2 implements FilterContainer {
 
   protected final Server webServer;
 
-  private final List<Connector> listeners = Lists.newArrayList();
+  private final HandlerCollection handlers;
+
+  private final List<ServerConnector> listeners = Lists.newArrayList();
 
   protected final WebAppContext webAppContext;
   protected final boolean findPort;
-  protected final Map<Context, Boolean> defaultContexts =
+  private final Map<ServletContextHandler, Boolean> defaultContexts =
       new HashMap<>();
   protected final List<String> filterNames = new ArrayList<>();
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
@@ -327,49 +333,59 @@ public final class HttpServer2 implements FilterContainer {
       }
 
       for (URI ep : endpoints) {
-        final Connector listener;
+        final ServerConnector connector;
         String scheme = ep.getScheme();
         if ("http".equals(scheme)) {
-          listener = HttpServer2.createDefaultChannelConnector();
+          connector =
+              HttpServer2.createDefaultChannelConnector(server.webServer);
         } else if ("https".equals(scheme)) {
-          listener = createHttpsChannelConnector();
+          connector = createHttpsChannelConnector(server.webServer);
 
         } else {
           throw new HadoopIllegalArgumentException(
               "unknown scheme for endpoint:" + ep);
         }
-        listener.setHost(ep.getHost());
-        listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
-        server.addListener(listener);
+        connector.setHost(ep.getHost());
+        connector.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
+        server.addListener(connector);
       }
       server.loadListeners();
       return server;
     }
 
-    private Connector createHttpsChannelConnector() {
-      SslSelectChannelConnector c = new SslSelectChannelConnectorSecure();
-      configureChannelConnector(c);
-
-      c.setNeedClientAuth(needsClientAuth);
-      c.setKeyPassword(keyPassword);
-
+    private ServerConnector createHttpsChannelConnector(Server server) {
+      ServerConnector conn = new ServerConnector(server);
+      HttpConfiguration httpConfig = new HttpConfiguration();
+      httpConfig.setRequestHeaderSize(JettyUtils.HEADER_SIZE);
+      httpConfig.setResponseHeaderSize(JettyUtils.HEADER_SIZE);
+      httpConfig.setSecureScheme("https");
+      httpConfig.addCustomizer(new SecureRequestCustomizer());
+      ConnectionFactory connFactory = new HttpConnectionFactory(httpConfig);
+      conn.addConnectionFactory(connFactory);
+      configureChannelConnector(conn);
+
+      SslContextFactory sslContextFactory = new SslContextFactory();
+      sslContextFactory.setNeedClientAuth(needsClientAuth);
+      sslContextFactory.setKeyManagerPassword(keyPassword);
       if (keyStore != null) {
-        c.setKeystore(keyStore);
-        c.setKeystoreType(keyStoreType);
-        c.setPassword(keyStorePassword);
+        sslContextFactory.setKeyStorePath(keyStore);
+        sslContextFactory.setKeyStoreType(keyStoreType);
+        sslContextFactory.setKeyStorePassword(keyStorePassword);
       }
-
       if (trustStore != null) {
-        c.setTruststore(trustStore);
-        c.setTruststoreType(trustStoreType);
-        c.setTrustPassword(trustStorePassword);
+        sslContextFactory.setTrustStorePath(trustStore);
+        sslContextFactory.setTrustStoreType(trustStoreType);
+        sslContextFactory.setTrustStorePassword(trustStorePassword);
       }
-
       if(null != excludeCiphers && !excludeCiphers.isEmpty()) {
-        c.setExcludeCipherSuites(excludeCiphers.split(","));
+        sslContextFactory.setExcludeCipherSuites(excludeCiphers.split(","));
         LOG.info("Excluded Cipher List:" + excludeCiphers);
       }
-      return c;
+
+      conn.addFirstConnectionFactory(new SslConnectionFactory(sslContextFactory,
+          HttpVersion.HTTP_1_1.asString()));
+
+      return conn;
     }
   }
 
@@ -377,6 +393,7 @@ public final class HttpServer2 implements FilterContainer {
     final String appDir = getWebAppsPath(b.name);
     this.webServer = new Server();
     this.adminsAcl = b.adminsAcl;
+    this.handlers = new HandlerCollection();
     this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
     this.xFrameOptionIsEnabled = b.xFrameEnabled;
     this.xFrameOption = b.xFrameOption;
@@ -406,36 +423,33 @@ public final class HttpServer2 implements FilterContainer {
     int maxThreads = conf.getInt(HTTP_MAX_THREADS, -1);
     // If HTTP_MAX_THREADS is not configured, QueueThreadPool() will use the
     // default value (currently 250).
-    QueuedThreadPool threadPool = maxThreads == -1 ? new QueuedThreadPool()
-        : new QueuedThreadPool(maxThreads);
+
+    QueuedThreadPool threadPool = (QueuedThreadPool) webServer.getThreadPool();
     threadPool.setDaemon(true);
-    webServer.setThreadPool(threadPool);
+    if (maxThreads != -1) {
+      threadPool.setMaxThreads(maxThreads);
+    }
 
     SessionManager sm = webAppContext.getSessionHandler().getSessionManager();
     if (sm instanceof AbstractSessionManager) {
       AbstractSessionManager asm = (AbstractSessionManager)sm;
       asm.setHttpOnly(true);
-      asm.setSecureCookies(true);
+      asm.getSessionCookieConfig().setSecure(true);
     }
 
     ContextHandlerCollection contexts = new ContextHandlerCollection();
     RequestLog requestLog = HttpRequestLog.getRequestLog(name);
 
+    handlers.addHandler(contexts);
     if (requestLog != null) {
       RequestLogHandler requestLogHandler = new RequestLogHandler();
       requestLogHandler.setRequestLog(requestLog);
-      HandlerCollection handlers = new HandlerCollection();
-      handlers.setHandlers(new Handler[] {contexts, requestLogHandler});
-      webServer.setHandler(handlers);
-    } else {
-      webServer.setHandler(contexts);
+      handlers.addHandler(requestLogHandler);
     }
-
+    handlers.addHandler(webAppContext);
     final String appDir = getWebAppsPath(name);
-
-    webServer.addHandler(webAppContext);
-
     addDefaultApps(contexts, appDir, conf);
+    webServer.setHandler(handlers);
 
     Map<String, String> xFrameParams = new HashMap<>();
     xFrameParams.put(X_FRAME_ENABLED,
@@ -461,7 +475,7 @@ public final class HttpServer2 implements FilterContainer {
     }
   }
 
-  private void addListener(Connector connector) {
+  private void addListener(ServerConnector connector) {
     listeners.add(connector);
   }
 
@@ -507,16 +521,14 @@ public final class HttpServer2 implements FilterContainer {
     return prop;
   }
 
-  private static void addNoCacheFilter(WebAppContext ctxt) {
+  private static void addNoCacheFilter(ServletContextHandler ctxt) {
     defineFilter(ctxt, NO_CACHE_FILTER, NoCacheFilter.class.getName(),
                  Collections.<String, String> emptyMap(), new String[] { "/*" });
   }
 
-  private static void configureChannelConnector(SelectChannelConnector c) {
-    c.setLowResourceMaxIdleTime(10000);
+  private static void configureChannelConnector(ServerConnector c) {
+    c.setIdleTimeout(10000);
     c.setAcceptQueueSize(128);
-    c.setResolveNames(false);
-    c.setUseDirectBuffers(false);
     if(Shell.WINDOWS) {
       // result of setting the SO_REUSEADDR flag is different on Windows
       // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
@@ -524,14 +536,18 @@ public final class HttpServer2 implements FilterContainer {
       // the same port with indeterminate routing of incoming requests to them
       c.setReuseAddress(false);
     }
-    c.setHeaderBufferSize(1024*64);
   }
 
   @InterfaceAudience.Private
-  public static Connector createDefaultChannelConnector() {
-    SelectChannelConnector ret = new SelectChannelConnector();
-    configureChannelConnector(ret);
-    return ret;
+  public static ServerConnector createDefaultChannelConnector(Server server) {
+    ServerConnector conn = new ServerConnector(server);
+    HttpConfiguration httpConfig = new HttpConfiguration();
+    httpConfig.setRequestHeaderSize(JettyUtils.HEADER_SIZE);
+    httpConfig.setResponseHeaderSize(JettyUtils.HEADER_SIZE);
+    ConnectionFactory connFactory = new HttpConnectionFactory(httpConfig);
+    conn.addConnectionFactory(connFactory);
+    configureChannelConnector(conn);
+    return conn;
   }
 
   /** Get an array of FilterConfiguration specified in the conf */
@@ -567,7 +583,8 @@ public final class HttpServer2 implements FilterContainer {
         CommonConfigurationKeys.HADOOP_HTTP_LOGS_ENABLED,
         CommonConfigurationKeys.HADOOP_HTTP_LOGS_ENABLED_DEFAULT);
     if (logDir != null && logsEnabled) {
-      Context logContext = new Context(parent, "/logs");
+      ServletContextHandler logContext =
+          new ServletContextHandler(parent, "/logs");
       logContext.setResourceBase(logDir);
       logContext.addServlet(AdminAuthorizedServlet.class, "/*");
       if (conf.getBoolean(
@@ -575,8 +592,7 @@ public final class HttpServer2 implements FilterContainer {
           CommonConfigurationKeys.DEFAULT_HADOOP_JETTY_LOGS_SERVE_ALIASES)) {
         @SuppressWarnings("unchecked")
         Map<String, String> params = logContext.getInitParams();
-        params.put(
-            "org.mortbay.jetty.servlet.Default.aliases", "true");
+        params.put("org.eclipse.jetty.servlet.Default.aliases", "true");
       }
       logContext.setDisplayName("logs");
       SessionHandler handler = new SessionHandler();
@@ -584,34 +600,37 @@ public final class HttpServer2 implements FilterContainer {
       if (sm instanceof AbstractSessionManager) {
         AbstractSessionManager asm = (AbstractSessionManager) sm;
         asm.setHttpOnly(true);
-        asm.setSecureCookies(true);
+        asm.getSessionCookieConfig().setSecure(true);
       }
       logContext.setSessionHandler(handler);
       setContextAttributes(logContext, conf);
-      addNoCacheFilter(webAppContext);
+      addNoCacheFilter(logContext);
       defaultContexts.put(logContext, true);
     }
     // set up the context for "/static/*"
-    Context staticContext = new Context(parent, "/static");
+    ServletContextHandler staticContext =
+        new ServletContextHandler(parent, "/static");
     staticContext.setResourceBase(appDir + "/static");
     staticContext.addServlet(DefaultServlet.class, "/*");
     staticContext.setDisplayName("static");
     @SuppressWarnings("unchecked")
     Map<String, String> params = staticContext.getInitParams();
-    params.put("org.mortbay.jetty.servlet.Default.dirAllowed", "false");
+    params.put("org.eclipse.jetty.servlet.Default.dirAllowed", "false");
+    params.put("org.eclipse.jetty.servlet.Default.gzip", "true");
     SessionHandler handler = new SessionHandler();
     SessionManager sm = handler.getSessionManager();
     if (sm instanceof AbstractSessionManager) {
       AbstractSessionManager asm = (AbstractSessionManager) sm;
       asm.setHttpOnly(true);
-      asm.setSecureCookies(true);
+      asm.getSessionCookieConfig().setSecure(true);
     }
     staticContext.setSessionHandler(handler);
     setContextAttributes(staticContext, conf);
     defaultContexts.put(staticContext, true);
   }
 
-  private void setContextAttributes(Context context, Configuration conf) {
+  private void setContextAttributes(ServletContextHandler context,
+                                    Configuration conf) {
     context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
   }
@@ -627,9 +646,9 @@ public final class HttpServer2 implements FilterContainer {
     addServlet("conf", "/conf", ConfServlet.class);
   }
 
-  public void addContext(Context ctxt, boolean isFiltered) {
-    webServer.addHandler(ctxt);
-    addNoCacheFilter(webAppContext);
+  public void addContext(ServletContextHandler ctxt, boolean isFiltered) {
+    handlers.addHandler(ctxt);
+    addNoCacheFilter(ctxt);
     defaultContexts.put(ctxt, isFiltered);
   }
 
@@ -691,7 +710,7 @@ public final class HttpServer2 implements FilterContainer {
    * protect with Kerberos authentication.
    * Note: This method is to be used for adding servlets that facilitate
    * internal communication and not for user facing functionality. For
-   +   * servlets added using this method, filters (except internal Kerberos
+   * servlets added using this method, filters (except internal Kerberos
    * filters) are not enabled.
    *
    * @param name The name of the servlet (can be passed as null)
@@ -705,19 +724,58 @@ public final class HttpServer2 implements FilterContainer {
     if (name != null) {
       holder.setName(name);
     }
+    // Jetty doesn't like the same path spec mapping to different servlets, so
+    // if there's already a mapping for this pathSpec, remove it and assume that
+    // the newest one is the one we want
+    final ServletMapping[] servletMappings =
+        webAppContext.getServletHandler().getServletMappings();
+    for (int i = 0; i < servletMappings.length; i++) {
+      if (servletMappings[i].containsPathSpec(pathSpec)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Found existing " + servletMappings[i].getServletName() +
+              " servlet at path " + pathSpec + "; will replace mapping" +
+              " with " + holder.getName() + " servlet");
+        }
+        ServletMapping[] newServletMappings =
+            ArrayUtil.removeFromArray(servletMappings, servletMappings[i]);
+        webAppContext.getServletHandler()
+            .setServletMappings(newServletMappings);
+        break;
+      }
+    }
     webAppContext.addServlet(holder, pathSpec);
 
     if(requireAuth && UserGroupInformation.isSecurityEnabled()) {
-       LOG.info("Adding Kerberos (SPNEGO) filter to " + name);
-       ServletHandler handler = webAppContext.getServletHandler();
-       FilterMapping fmap = new FilterMapping();
-       fmap.setPathSpec(pathSpec);
-       fmap.setFilterName(SPNEGO_FILTER);
-       fmap.setDispatches(Handler.ALL);
-       handler.addFilterMapping(fmap);
+      LOG.info("Adding Kerberos (SPNEGO) filter to " + name);
+      ServletHandler handler = webAppContext.getServletHandler();
+      FilterMapping fmap = new FilterMapping();
+      fmap.setPathSpec(pathSpec);
+      fmap.setFilterName(SPNEGO_FILTER);
+      fmap.setDispatches(FilterMapping.ALL);
+      handler.addFilterMapping(fmap);
     }
   }
 
+  /**
+   * Add the given handler to the front of the list of handlers.
+   *
+   * @param handler The handler to add
+   */
+  public void addHandlerAtFront(Handler handler) {
+    Handler[] h = ArrayUtil.prependToArray(
+        handler, this.handlers.getHandlers(), Handler.class);
+    handlers.setHandlers(h);
+  }
+
+  /**
+   * Add the given handler to the end of the list of handlers.
+   *
+   * @param handler The handler to add
+   */
+  public void addHandlerAtEnd(Handler handler) {
+    handlers.addHandler(handler);
+  }
+
   @Override
   public void addFilter(String name, String classname,
       Map<String, String> parameters) {
@@ -727,12 +785,14 @@ public final class HttpServer2 implements FilterContainer {
     FilterMapping fmap = getFilterMapping(name, USER_FACING_URLS);
     defineFilter(webAppContext, filterHolder, fmap);
     LOG.info(
-        "Added filter " + name + " (class=" + classname + ") to context " + webAppContext.getDisplayName());
+        "Added filter " + name + " (class=" + classname + ") to context "
+            + webAppContext.getDisplayName());
     final String[] ALL_URLS = { "/*" };
     fmap = getFilterMapping(name, ALL_URLS);
-    for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
+    for (Map.Entry<ServletContextHandler, Boolean> e
+        : defaultContexts.entrySet()) {
       if (e.getValue()) {
-        Context ctx = e.getKey();
+        ServletContextHandler ctx = e.getKey();
         defineFilter(ctx, filterHolder, fmap);
         LOG.info("Added filter " + name + " (class=" + classname
             + ") to context " + ctx.getDisplayName());
@@ -748,7 +808,7 @@ public final class HttpServer2 implements FilterContainer {
     FilterHolder filterHolder = getFilterHolder(name, classname, parameters);
     FilterMapping fmap = getFilterMapping(name, ALL_URLS);
     defineFilter(webAppContext, filterHolder, fmap);
-    for (Context ctx : defaultContexts.keySet()) {
+    for (ServletContextHandler ctx : defaultContexts.keySet()) {
       defineFilter(ctx, filterHolder, fmap);
     }
     LOG.info("Added global filter '" + name + "' (class=" + classname + ")");
@@ -757,7 +817,7 @@ public final class HttpServer2 implements FilterContainer {
   /**
    * Define a filter for a context and set up default url mappings.
    */
-  public static void defineFilter(Context ctx, String name,
+  public static void defineFilter(ServletContextHandler ctx, String name,
       String classname, Map<String,String> parameters, String[] urls) {
     FilterHolder filterHolder = getFilterHolder(name, classname, parameters);
     FilterMapping fmap = getFilterMapping(name, urls);
@@ -767,8 +827,8 @@ public final class HttpServer2 implements FilterContainer {
   /**
    * Define a filter for a context and set up default url mappings.
    */
-  private static void defineFilter(Context ctx, FilterHolder holder,
-      FilterMapping fmap) {
+  private static void defineFilter(ServletContextHandler ctx,
+                                   FilterHolder holder, FilterMapping fmap) {
     ServletHandler handler = ctx.getServletHandler();
     handler.addFilter(holder, fmap);
   }
@@ -776,7 +836,7 @@ public final class HttpServer2 implements FilterContainer {
   private static FilterMapping getFilterMapping(String name, String[] urls) {
     FilterMapping fmap = new FilterMapping();
     fmap.setPathSpecs(urls);
-    fmap.setDispatches(Handler.ALL);
+    fmap.setDispatches(FilterMapping.ALL);
     fmap.setFilterName(name);
     return fmap;
   }
@@ -786,7 +846,9 @@ public final class HttpServer2 implements FilterContainer {
     FilterHolder holder = new FilterHolder();
     holder.setName(name);
     holder.setClassName(classname);
-    holder.setInitParameters(parameters);
+    if (parameters != null) {
+      holder.setInitParameters(parameters);
+    }
     return holder;
   }
 
@@ -796,13 +858,13 @@ public final class HttpServer2 implements FilterContainer {
    * @param webAppCtx The WebApplicationContext to add to
    */
   protected void addFilterPathMapping(String pathSpec,
-      Context webAppCtx) {
+      ServletContextHandler webAppCtx) {
     ServletHandler handler = webAppCtx.getServletHandler();
     for(String name : filterNames) {
       FilterMapping fmap = new FilterMapping();
       fmap.setPathSpec(pathSpec);
       fmap.setFilterName(name);
-      fmap.setDispatches(Handler.ALL);
+      fmap.setDispatches(FilterMapping.ALL);
       handler.addFilterMapping(fmap);
     }
   }
@@ -841,23 +903,23 @@ public final class HttpServer2 implements FilterContainer {
    */
   @Deprecated
   public int getPort() {
-    return webServer.getConnectors()[0].getLocalPort();
+    return ((ServerConnector)webServer.getConnectors()[0]).getLocalPort();
   }
 
   /**
    * Get the address that corresponds to a particular connector.
    *
    * @return the corresponding address for the connector, or null if there's no
-   *         such connector or the connector is not bounded.
+   *         such connector or the connector is not bounded or was closed.
    */
   public InetSocketAddress getConnectorAddress(int index) {
     Preconditions.checkArgument(index >= 0);
     if (index > webServer.getConnectors().length)
       return null;
 
-    Connector c = webServer.getConnectors()[index];
-    if (c.getLocalPort() == -1) {
-      // The connector is not bounded
+    ServerConnector c = (ServerConnector)webServer.getConnectors()[index];
+    if (c.getLocalPort() == -1 || c.getLocalPort() == -2) {
+      // The connector is not bounded or was closed
       return null;
     }
 
@@ -907,8 +969,8 @@ public final class HttpServer2 implements FilterContainer {
         throw ex;
       }
       // Make sure there is no handler failures.
-      Handler[] handlers = webServer.getHandlers();
-      for (Handler handler : handlers) {
+      Handler[] hs = webServer.getHandlers();
+      for (Handler handler : hs) {
         if (handler.isFailed()) {
           throw new IOException(
               "Problem in starting http server. Server handlers failed");
@@ -944,9 +1006,10 @@ public final class HttpServer2 implements FilterContainer {
    * @throws Exception
    */
   void openListeners() throws Exception {
-    for (Connector listener : listeners) {
-      if (listener.getLocalPort() != -1) {
-        // This listener is either started externally or has been bound
+    for (ServerConnector listener : listeners) {
+      if (listener.getLocalPort() != -1 && listener.getLocalPort() != -2) {
+        // This listener is either started externally or has been bound or was
+        // closed
         continue;
       }
       int port = listener.getPort();
@@ -978,7 +1041,7 @@ public final class HttpServer2 implements FilterContainer {
    */
   public void stop() throws Exception {
     MultiException exception = null;
-    for (Connector c : listeners) {
+    for (ServerConnector c : listeners) {
       try {
         c.close();
       } catch (Exception e) {
@@ -1042,7 +1105,7 @@ public final class HttpServer2 implements FilterContainer {
         .append(isAlive() ? STATE_DESCRIPTION_ALIVE
                     : STATE_DESCRIPTION_NOT_LIVE)
         .append("), listening at:");
-    for (Connector l : listeners) {
+    for (ServerConnector l : listeners) {
       sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
     }
     return sb.toString();
@@ -1300,10 +1363,10 @@ public final class HttpServer2 implements FilterContainer {
      */
     private String inferMimeType(ServletRequest request) {
       String path = ((HttpServletRequest)request).getRequestURI();
-      ContextHandler.SContext sContext = (ContextHandler.SContext)config.getServletContext();
-      MimeTypes mimes = sContext.getContextHandler().getMimeTypes();
-      Buffer mimeBuffer = mimes.getMimeByExtension(path);
-      return (mimeBuffer == null) ? null : mimeBuffer.toString();
+      ServletContextHandler.Context sContext =
+          (ServletContextHandler.Context)config.getServletContext();
+      String mime = sContext.getMimeType(path);
+      return (mime == null) ? null : mime;
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/JettyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/JettyUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/JettyUtils.java
new file mode 100644
index 0000000..29c0930
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/JettyUtils.java
@@ -0,0 +1,35 @@
+/**
+ * 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.http;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Contains utility methods and constants relating to Jetty.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class JettyUtils {
+  public static final String UTF_8 = "charset=utf-8";
+  public static final int HEADER_SIZE = 1024 * 64;
+
+  private JettyUtils() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SslSelectChannelConnectorSecure.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SslSelectChannelConnectorSecure.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SslSelectChannelConnectorSecure.java
deleted file mode 100644
index 7de689b..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SslSelectChannelConnectorSecure.java
+++ /dev/null
@@ -1,58 +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.security.ssl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import javax.net.ssl.SSLEngine;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.mortbay.jetty.security.SslSelectChannelConnector;
-
-/**
- * This subclass of the Jetty SslSelectChannelConnector exists solely to
- * control the TLS protocol versions allowed.  This is fallout from the
- * POODLE vulnerability (CVE-2014-3566), which requires that SSLv3 be disabled.
- * Only TLS 1.0 and later protocols are allowed.
- */
-@InterfaceAudience.Private
-public class SslSelectChannelConnectorSecure extends SslSelectChannelConnector {
-
-  public SslSelectChannelConnectorSecure() {
-    super();
-  }
-
-  /**
-   * Disable SSLv3 protocol.
-   */
-  @Override
-  protected SSLEngine createSSLEngine() throws IOException {
-    SSLEngine engine = super.createSSLEngine();
-    ArrayList<String> nonSSLProtocols = new ArrayList<String>();
-    for (String p : engine.getEnabledProtocols()) {
-      if (!p.contains("SSLv3")) {
-        nonSSLProtocols.add(p);
-      }
-    }
-    engine.setEnabledProtocols(nonSSLProtocols.toArray(
-        new String[nonSSLProtocols.size()]));
-    return engine;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
index 60035be..53089ed 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
@@ -31,7 +31,7 @@ import javax.ws.rs.core.HttpHeaders;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
index 35fd9be..f0c00c4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
@@ -32,7 +32,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 /**
  * <p>
@@ -797,7 +797,7 @@ public abstract class FSMainOperationsBaseTest extends FileSystemTestHelper {
       rename(src, dst, false, false, false, Rename.NONE);
       Assert.fail("Should throw FileNotFoundException");
     } catch (IOException e) {
-      Log.info("XXX", e);
+      Log.getLog().info("XXX", e);
       Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
index 81ca210..866c03e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.util.Shell;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 
 /**
@@ -84,7 +84,7 @@ public class ViewFileSystemTestSetup {
 
     FileSystem fsView = FileSystem.get(FsConstants.VIEWFS_URI, conf);
     fsView.setWorkingDirectory(new Path(wdDir)); // in case testdir relative to wd.
-    Log.info("Working dir is: " + fsView.getWorkingDirectory());
+    Log.getLog().info("Working dir is: " + fsView.getWorkingDirectory());
     return fsView;
   }
 
@@ -118,12 +118,12 @@ public class ViewFileSystemTestSetup {
     } else { // home dir is at root. Just link the home dir itse
       URI linkTarget = fsTarget.makeQualified(new Path(homeDir)).toUri();
       ConfigUtil.addLink(conf, homeDir, linkTarget);
-      Log.info("Added link for home dir " + homeDir + "->" + linkTarget);
+      Log.getLog().info("Added link for home dir " + homeDir + "->" + linkTarget);
     }
     // Now set the root of the home dir for viewfs
     String homeDirRoot = fsTarget.getHomeDirectory().getParent().toUri().getPath();
     ConfigUtil.setHomeDirConf(conf, homeDirRoot);
-    Log.info("Home dir base for viewfs" + homeDirRoot);  
+    Log.getLog().info("Home dir base for viewfs" + homeDirRoot);
   }
   
   /*
@@ -138,7 +138,7 @@ public class ViewFileSystemTestSetup {
     String firstComponent = path.substring(0, indexOfEnd);
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
-    Log.info("Added link for " + info + " " 
+    Log.getLog().info("Added link for " + info + " "
         + firstComponent + "->" + linkTarget);    
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
index 92bcbc3..9b7e17f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.util.Shell;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 
 /**
@@ -82,7 +82,7 @@ public class ViewFsTestSetup {
     
     FileContext fc = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
     fc.setWorkingDirectory(new Path(wdDir)); // in case testdir relative to wd.
-    Log.info("Working dir is: " + fc.getWorkingDirectory());
+    Log.getLog().info("Working dir is: " + fc.getWorkingDirectory());
     //System.out.println("SRCOfTests = "+ getTestRootPath(fc, "test"));
     //System.out.println("TargetOfTests = "+ targetOfTests.toUri());
     return fc;
@@ -107,12 +107,12 @@ public class ViewFsTestSetup {
     } else { // home dir is at root. Just link the home dir itse
       URI linkTarget = fsTarget.makeQualified(new Path(homeDir)).toUri();
       ConfigUtil.addLink(conf, homeDir, linkTarget);
-      Log.info("Added link for home dir " + homeDir + "->" + linkTarget);
+      Log.getLog().info("Added link for home dir " + homeDir + "->" + linkTarget);
     }
     // Now set the root of the home dir for viewfs
     String homeDirRoot = fsTarget.getHomeDirectory().getParent().toUri().getPath();
     ConfigUtil.setHomeDirConf(conf, homeDirRoot);
-    Log.info("Home dir base for viewfs" + homeDirRoot);  
+    Log.getLog().info("Home dir base for viewfs" + homeDirRoot);
   }
   
   /*
@@ -128,7 +128,7 @@ public class ViewFsTestSetup {
     String firstComponent = path.substring(0, indexOfEnd);
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
-    Log.info("Added link for " + info + " " 
+    Log.getLog().info("Added link for " + info + " "
         + firstComponent + "->" + linkTarget);    
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
index c51f1e8..44338da 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 import javax.servlet.*;
 import javax.servlet.http.HttpServletResponse;
@@ -33,6 +33,7 @@ import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
 import java.net.HttpCookie;
+import java.util.HashMap;
 import java.util.List;
 
 public class TestAuthenticationSessionCookie {
@@ -71,7 +72,7 @@ public class TestAuthenticationSessionCookie {
     @Override
     public void initFilter(FilterContainer container, Configuration conf) {
       container.addFilter("DummyAuth", DummyAuthenticationFilter.class
-              .getName(), null);
+              .getName(), new HashMap<>());
     }
   }
 
@@ -93,7 +94,7 @@ public class TestAuthenticationSessionCookie {
     @Override
     public void initFilter(FilterContainer container, Configuration conf) {
       container.addFilter("Dummy2Auth", Dummy2AuthenticationFilter.class
-              .getName(), null);
+              .getName(), new HashMap<>());
     }
   }
 
@@ -149,7 +150,7 @@ public class TestAuthenticationSessionCookie {
     String header = conn.getHeaderField("Set-Cookie");
     List<HttpCookie> cookies = HttpCookie.parse(header);
     Assert.assertTrue(!cookies.isEmpty());
-    Log.info(header);
+    Log.getLog().info(header);
     Assert.assertFalse(header.contains("; Expires="));
     Assert.assertTrue("token".equals(cookies.get(0).getValue()));
   }
@@ -171,7 +172,7 @@ public class TestAuthenticationSessionCookie {
     String header = conn.getHeaderField("Set-Cookie");
     List<HttpCookie> cookies = HttpCookie.parse(header);
     Assert.assertTrue(!cookies.isEmpty());
-    Log.info(header);
+    Log.getLog().info(header);
     Assert.assertTrue(header.contains("; Expires="));
     Assert.assertTrue("token".equals(cookies.get(0).getValue()));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java
index 23e0d3e..212807f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java
@@ -18,9 +18,9 @@
 package org.apache.hadoop.http;
 
 import org.apache.log4j.Logger;
+import org.eclipse.jetty.server.NCSARequestLog;
+import org.eclipse.jetty.server.RequestLog;
 import org.junit.Test;
-import org.mortbay.jetty.NCSARequestLog;
-import org.mortbay.jetty.RequestLog;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
index 98f0a0e..a36e8ca 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.util.ajax.JSON;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -36,8 +38,6 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
-import org.mortbay.jetty.Connector;
-import org.mortbay.util.ajax.JSON;
 
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
@@ -50,6 +50,7 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.HttpURLConnection;
@@ -81,6 +82,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     public void doGet(HttpServletRequest request, 
                       HttpServletResponse response
                       ) throws ServletException, IOException {
+      response.setContentType(MediaType.TEXT_PLAIN + "; " + JettyUtils.UTF_8);
       PrintWriter out = response.getWriter();
       Map<String, String[]> params = request.getParameterMap();
       SortedSet<String> keys = new TreeSet<String>(params.keySet());
@@ -108,6 +110,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     public void doGet(HttpServletRequest request, 
                       HttpServletResponse response
                       ) throws ServletException, IOException {
+      response.setContentType(MediaType.TEXT_PLAIN + "; " + JettyUtils.UTF_8);
       PrintWriter out = response.getWriter();
       SortedSet<String> sortedKeys = new TreeSet<String>();
       Enumeration<String> keys = request.getParameterNames();
@@ -130,7 +133,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     public void doGet(HttpServletRequest request, 
                       HttpServletResponse response
                       ) throws ServletException, IOException {
-      response.setContentType("text/html");
+      response.setContentType(MediaType.TEXT_HTML + "; " + JettyUtils.UTF_8);
       PrintWriter out = response.getWriter();
       out.print("hello world");
       out.close();
@@ -222,7 +225,8 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     conn = (HttpURLConnection)servletUrl.openConnection();
     conn.connect();
     assertEquals(200, conn.getResponseCode());
-    assertEquals("text/plain; charset=utf-8", conn.getContentType());
+    assertEquals(MediaType.TEXT_PLAIN + ";" + JettyUtils.UTF_8,
+        conn.getContentType());
 
     // We should ignore parameters for mime types - ie a parameter
     // ending in .css should not change mime type
@@ -230,14 +234,16 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     conn = (HttpURLConnection)servletUrl.openConnection();
     conn.connect();
     assertEquals(200, conn.getResponseCode());
-    assertEquals("text/plain; charset=utf-8", conn.getContentType());
+    assertEquals(MediaType.TEXT_PLAIN + ";" + JettyUtils.UTF_8,
+        conn.getContentType());
 
     // Servlets that specify text/html should get that content type
     servletUrl = new URL(baseUrl, "/htmlcontent");
     conn = (HttpURLConnection)servletUrl.openConnection();
     conn.connect();
     assertEquals(200, conn.getResponseCode());
-    assertEquals("text/html; charset=utf-8", conn.getContentType());
+    assertEquals(MediaType.TEXT_HTML + ";" + JettyUtils.UTF_8,
+        conn.getContentType());
   }
 
   @Test
@@ -488,7 +494,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
 
   @SuppressWarnings("unchecked")
   private static Map<String, Object> parse(String jsonString) {
-    return (Map<String, Object>)JSON.parse(jsonString);
+    return (Map<String, Object>) JSON.parse(jsonString);
   }
 
   @Test public void testJersey() throws Exception {
@@ -592,7 +598,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
       // not bound, ephemeral should return requested port (0 for ephemeral)
       List<?> listeners = (List<?>) Whitebox.getInternalState(server,
           "listeners");
-      Connector listener = (Connector) listeners.get(0);
+      ServerConnector listener = (ServerConnector)listeners.get(0);
 
       assertEquals(port, listener.getPort());
       // verify hostname is what was given

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java
index b0fadcf..f58c230 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java
@@ -146,9 +146,11 @@ public class TestServletFilter extends HttpServerFunctionalTest {
   }
   
   static public class ErrorFilter extends SimpleFilter {
+    static final String EXCEPTION_MESSAGE =
+        "Throwing the exception from Filter init";
     @Override
     public void init(FilterConfig arg0) throws ServletException {
-      throw new ServletException("Throwing the exception from Filter init");
+      throw new ServletException(EXCEPTION_MESSAGE);
     }
 
     /** Configuration for the filter */
@@ -174,7 +176,8 @@ public class TestServletFilter extends HttpServerFunctionalTest {
       http.start();
       fail("expecting exception");
     } catch (IOException e) {
-      assertTrue( e.getMessage().contains("Problem in starting http server. Server handlers failed"));
+      assertEquals("Problem starting http server", e.getMessage());
+      assertEquals(ErrorFilter.EXCEPTION_MESSAGE, e.getCause().getMessage());
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/resource/JerseyResource.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/resource/JerseyResource.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/resource/JerseyResource.java
index f1313e2..607d17f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/resource/JerseyResource.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/resource/JerseyResource.java
@@ -32,7 +32,8 @@ import javax.ws.rs.core.Response;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.mortbay.util.ajax.JSON;
+import org.apache.hadoop.http.JettyUtils;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * A simple Jersey resource class TestHttpServer.
@@ -48,7 +49,7 @@ public class JerseyResource {
 
   @GET
   @Path("{" + PATH + ":.*}")
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response get(
       @PathParam(PATH) @DefaultValue("UNKNOWN_" + PATH) final String path,
       @QueryParam(OP) @DefaultValue("UNKNOWN_" + OP) final String op

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
index 73562b5..44241de 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
@@ -31,22 +31,22 @@ import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHand
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.jetty.AbstractConnector;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.FilterHolder;
-import org.mortbay.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletHolder;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
+import javax.servlet.DispatcherType;
 import javax.servlet.Filter;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
@@ -66,6 +66,7 @@ import java.net.URL;
 import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -177,7 +178,7 @@ public class TestWebDelegationToken {
   protected Server createJettyServer() {
     try {
       jetty = new Server(0);
-      jetty.getConnectors()[0].setHost("localhost");
+      ((ServerConnector)jetty.getConnectors()[0]).setHost("localhost");
       return jetty;
     } catch (Exception ex) {
       throw new RuntimeException("Could not setup Jetty: " + ex.getMessage(),
@@ -186,7 +187,7 @@ public class TestWebDelegationToken {
   }
 
   protected String getJettyURL() {
-    Connector c = jetty.getConnectors()[0];
+    ServerConnector c = (ServerConnector)jetty.getConnectors()[0];
     return "http://" + c.getHost() + ":" + c.getLocalPort();
   }
 
@@ -217,10 +218,11 @@ public class TestWebDelegationToken {
   @Test
   public void testRawHttpCalls() throws Exception {
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(AFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(AFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(PingServlet.class), "/bar");
     try {
       jetty.start();
@@ -337,10 +339,11 @@ public class TestWebDelegationToken {
   private void testDelegationTokenAuthenticatorCalls(final boolean useQS)
       throws Exception {
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(AFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(AFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(PingServlet.class), "/bar");
 
     try {
@@ -446,10 +449,11 @@ public class TestWebDelegationToken {
     DummyDelegationTokenSecretManager secretMgr
         = new DummyDelegationTokenSecretManager();
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(AFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(AFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(PingServlet.class), "/bar");
     try {
       secretMgr.startThreads();
@@ -525,10 +529,11 @@ public class TestWebDelegationToken {
   private void testDelegationTokenAuthenticatedURLWithNoDT(
       Class<? extends Filter> filterClass)  throws Exception {
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(filterClass), "/*", 0);
+    context.addFilter(new FilterHolder(filterClass), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(UserServlet.class), "/bar");
 
     try {
@@ -594,10 +599,11 @@ public class TestWebDelegationToken {
   public void testFallbackToPseudoDelegationTokenAuthenticator()
       throws Exception {
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(UserServlet.class), "/bar");
 
     try {
@@ -745,11 +751,11 @@ public class TestWebDelegationToken {
     Assert.assertTrue(testDir.mkdirs());
     MiniKdc kdc = new MiniKdc(MiniKdc.createConf(), testDir);
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    ((AbstractConnector)jetty.getConnectors()[0]).setResolveNames(true);
-    context.addFilter(new FilterHolder(KDTAFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(KDTAFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(UserServlet.class), "/bar");
     try {
       kdc.start();
@@ -824,10 +830,11 @@ public class TestWebDelegationToken {
   @Test
   public void testProxyUser() throws Exception {
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(UserServlet.class), "/bar");
 
     try {
@@ -921,10 +928,11 @@ public class TestWebDelegationToken {
   @Test
   public void testHttpUGI() throws Exception {
     final Server jetty = createJettyServer();
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
-    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(UGIServlet.class), "/bar");
 
     try {
@@ -980,12 +988,12 @@ public class TestWebDelegationToken {
   @Test
   public void testIpaddressCheck() throws Exception {
     final Server jetty = createJettyServer();
-    ((AbstractConnector)jetty.getConnectors()[0]).setResolveNames(true);
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     jetty.setHandler(context);
 
-    context.addFilter(new FilterHolder(IpAddressBasedPseudoDTAFilter.class), "/*", 0);
+    context.addFilter(new FilterHolder(IpAddressBasedPseudoDTAFilter.class), "/*",
+        EnumSet.of(DispatcherType.REQUEST));
     context.addServlet(new ServletHolder(UGIServlet.class), "/bar");
 
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-kms/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/pom.xml b/hadoop-common-project/hadoop-kms/pom.xml
index f65e94a..3eaaf52 100644
--- a/hadoop-common-project/hadoop-kms/pom.xml
+++ b/hadoop-common-project/hadoop-kms/pom.xml
@@ -80,12 +80,12 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -111,7 +111,7 @@
         </exclusion>
         <exclusion>
           <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
         <exclusion>
           <groupId>javax.servlet</groupId>
@@ -122,19 +122,19 @@
           <artifactId>jsp-api</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>jetty-util</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>jsp-api-2.1</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>servlet-api-2.5</artifactId>
         </exclusion>
         <exclusion>
@@ -178,7 +178,7 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
       <scope>compile</scope>
     </dependency>


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


[31/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js.gz
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js.gz b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js.gz
deleted file mode 100644
index abdb4b1..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js.gz and /dev/null differ


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


[28/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMNMInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMNMInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMNMInfo.java
index f6af030..1b7ddd3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMNMInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMNMInfo.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * JMX bean listing statuses of all node managers.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 99440a8..2c61339 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -60,6 +60,7 @@ 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.http.JettyUtils;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -257,14 +258,16 @@ public class RMWebServices extends WebServices {
   }
 
   @GET
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ClusterInfo get() {
     return getClusterInfo();
   }
 
   @GET
   @Path("/info")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ClusterInfo getClusterInfo() {
     init();
     return new ClusterInfo(this.rm);
@@ -272,7 +275,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/metrics")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ClusterMetricsInfo getClusterMetricsInfo() {
     init();
     return new ClusterMetricsInfo(this.rm);
@@ -280,7 +284,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/scheduler")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public SchedulerTypeInfo getSchedulerInfo() {
     init();
     ResourceScheduler rs = rm.getResourceScheduler();
@@ -303,7 +308,8 @@ public class RMWebServices extends WebServices {
 
   @POST
   @Path("/scheduler/logs")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public String dumpSchedulerLogs(@FormParam("time") String time,
       @Context HttpServletRequest hsr) throws IOException {
     init();
@@ -340,7 +346,8 @@ public class RMWebServices extends WebServices {
    */
   @GET
   @Path("/nodes")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodesInfo getNodes(@QueryParam("states") String states) {
     init();
     ResourceScheduler sched = this.rm.getResourceScheduler();
@@ -376,7 +383,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/nodes/{nodeId}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodeInfo getNode(@PathParam("nodeId") String nodeId) {
     init();
     if (nodeId == null || nodeId.isEmpty()) {
@@ -405,7 +413,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppsInfo getApps(@Context HttpServletRequest hsr,
       @QueryParam("state") String stateQuery,
       @QueryParam("states") Set<String> statesQuery,
@@ -583,7 +592,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/scheduler/activities")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ActivitiesInfo getActivities(@Context HttpServletRequest hsr,
       @QueryParam("nodeId") String nodeId) {
     YarnScheduler scheduler = rm.getRMContext().getScheduler();
@@ -651,7 +661,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/scheduler/app-activities")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
       @QueryParam("appId") String appId, @QueryParam("maxTime") String time) {
     YarnScheduler scheduler = rm.getRMContext().getScheduler();
@@ -701,7 +712,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/appstatistics")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ApplicationStatisticsInfo getAppStatistics(
       @Context HttpServletRequest hsr,
       @QueryParam("states") Set<String> stateQueries,
@@ -788,7 +800,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppInfo getApp(@Context HttpServletRequest hsr,
       @PathParam("appid") String appId) {
     init();
@@ -802,7 +815,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppAttemptsInfo getAppAttempts(@Context HttpServletRequest hsr,
       @PathParam("appid") String appId) {
 
@@ -825,7 +839,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts/{appattemptid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo getAppAttempt(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId,
@@ -836,7 +851,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts/{appattemptid}/containers")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public ContainersInfo getContainers(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId,
@@ -847,7 +863,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts/{appattemptid}/containers/{containerid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public ContainerInfo getContainer(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId,
@@ -859,7 +876,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/state")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppState getAppState(@Context HttpServletRequest hsr,
       @PathParam("appid") String appId) throws AuthorizationException {
     init();
@@ -890,7 +908,8 @@ public class RMWebServices extends WebServices {
 
   @PUT
   @Path("/apps/{appid}/state")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response updateAppState(AppState targetState,
       @Context HttpServletRequest hsr, @PathParam("appid") String appId)
@@ -940,7 +959,8 @@ public class RMWebServices extends WebServices {
   
   @GET
   @Path("/get-node-to-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr)
       throws IOException {
     init();
@@ -960,7 +980,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/label-mappings")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public LabelsToNodesInfo getLabelsToNodes(
       @QueryParam("labels") Set<String> labels) throws IOException {
     init();
@@ -989,7 +1010,8 @@ public class RMWebServices extends WebServices {
 
   @POST
   @Path("/replace-node-to-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response replaceLabelsOnNodes(final NodeToLabelsEntryList newNodeToLabels,
       @Context HttpServletRequest hsr) throws IOException {
     Map<NodeId, Set<String>> nodeIdToLabels =
@@ -1006,7 +1028,8 @@ public class RMWebServices extends WebServices {
 
   @POST
   @Path("/nodes/{nodeId}/replace-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response replaceLabelsOnNode(
       @QueryParam("labels") Set<String> newNodeLabelsName,
       @Context HttpServletRequest hsr, @PathParam("nodeId") String nodeId)
@@ -1054,7 +1077,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/get-node-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodeLabelsInfo getClusterNodeLabels(@Context HttpServletRequest hsr) 
     throws IOException {
     init();
@@ -1068,7 +1092,8 @@ public class RMWebServices extends WebServices {
   
   @POST
   @Path("/add-node-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response addToClusterNodeLabels(final NodeLabelsInfo newNodeLabels,
       @Context HttpServletRequest hsr)
       throws Exception {
@@ -1099,7 +1124,8 @@ public class RMWebServices extends WebServices {
   
   @POST
   @Path("/remove-node-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response removeFromCluserNodeLabels(
       @QueryParam("labels") Set<String> oldNodeLabels,
       @Context HttpServletRequest hsr) throws Exception {
@@ -1129,7 +1155,8 @@ public class RMWebServices extends WebServices {
   
   @GET
   @Path("/nodes/{nodeId}/get-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodeLabelsInfo getLabelsOnNode(@Context HttpServletRequest hsr,
       @PathParam("nodeId") String nodeId) throws IOException {
     init();
@@ -1199,7 +1226,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/priority")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppPriority getAppPriority(@Context HttpServletRequest hsr,
       @PathParam("appid") String appId) throws AuthorizationException {
     init();
@@ -1227,7 +1255,8 @@ public class RMWebServices extends WebServices {
 
   @PUT
   @Path("/apps/{appid}/priority")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response updateApplicationPriority(AppPriority targetPriority,
       @Context HttpServletRequest hsr, @PathParam("appid") String appId)
@@ -1313,7 +1342,8 @@ public class RMWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/queue")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppQueue getAppQueue(@Context HttpServletRequest hsr,
       @PathParam("appid") String appId) throws AuthorizationException {
     init();
@@ -1340,7 +1370,8 @@ public class RMWebServices extends WebServices {
 
   @PUT
   @Path("/apps/{appid}/queue")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response updateAppQueue(AppQueue targetQueue,
       @Context HttpServletRequest hsr, @PathParam("appid") String appId)
@@ -1477,7 +1508,8 @@ public class RMWebServices extends WebServices {
    */
   @POST
   @Path("/apps/new-application")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response createNewApplication(@Context HttpServletRequest hsr)
       throws AuthorizationException, IOException, InterruptedException {
     init();
@@ -1514,7 +1546,8 @@ public class RMWebServices extends WebServices {
    */
   @POST
   @Path("/apps")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response submitApplication(ApplicationSubmissionContextInfo newApp,
       @Context HttpServletRequest hsr) throws AuthorizationException,
@@ -1780,7 +1813,8 @@ public class RMWebServices extends WebServices {
 
   @POST
   @Path("/delegation-token")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response postDelegationToken(DelegationToken tokenData,
       @Context HttpServletRequest hsr) throws AuthorizationException,
@@ -1798,7 +1832,8 @@ public class RMWebServices extends WebServices {
 
   @POST
   @Path("/delegation-token/expiration")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response
       postDelegationTokenExpiration(@Context HttpServletRequest hsr)
@@ -1916,7 +1951,8 @@ public class RMWebServices extends WebServices {
   // the logs can extract tokens which are meant to be secret
   @DELETE
   @Path("/delegation-token")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response cancelDelegationToken(@Context HttpServletRequest hsr)
       throws AuthorizationException, IOException, InterruptedException,
       Exception {
@@ -2004,7 +2040,8 @@ public class RMWebServices extends WebServices {
    */
   @POST
   @Path("/reservation/new-reservation")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response createNewReservation(@Context HttpServletRequest hsr)
     throws AuthorizationException, IOException, InterruptedException {
     init();
@@ -2059,7 +2096,8 @@ public class RMWebServices extends WebServices {
    */
   @POST
   @Path("/reservation/submit")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response submitReservation(
       ReservationSubmissionRequestInfo resContext,
@@ -2168,7 +2206,8 @@ public class RMWebServices extends WebServices {
    */
   @POST
   @Path("/reservation/update")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response updateReservation(ReservationUpdateRequestInfo resContext,
       @Context HttpServletRequest hsr) throws AuthorizationException,
@@ -2282,7 +2321,8 @@ public class RMWebServices extends WebServices {
    */
   @POST
   @Path("/reservation/delete")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response deleteReservation(ReservationDeleteRequestInfo resContext,
       @Context HttpServletRequest hsr) throws AuthorizationException,
@@ -2340,7 +2380,8 @@ public class RMWebServices extends WebServices {
    */
   @GET
   @Path("/reservation/list")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public Response listReservation(
           @QueryParam("queue") @DefaultValue("default") String queue,
           @QueryParam("reservation-id") @DefaultValue("") String reservationId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.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/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
index 2e2bef7..d4aa113 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 public class MockNM {
 
@@ -175,7 +175,7 @@ public class MockNM {
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>(1);
     containerStatusList.add(containerStatus);
-    Log.info("ContainerStatus: " + containerStatus);
+    Log.getLog().info("ContainerStatus: " + containerStatus);
     return nodeHeartbeat(containerStatusList,
         Collections.<Container>emptyList(), true, ++responseId);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.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/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
index 905a42c..47c053c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
@@ -162,7 +163,8 @@ public class TestRMHA {
         webResource.path("ws").path("v1").path("cluster").path("apps")
           .path(path).accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     assertEquals("incorrect number of elements", 1, json.length());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.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/reservation/planning/TestAlignedPlanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java
index ec305a2..4b01eb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 public class TestAlignedPlanner {
 
@@ -701,7 +701,7 @@ public class TestAlignedPlanner {
     // Initialize random seed
     long seed = rand.nextLong();
     rand.setSeed(seed);
-    Log.info("Running with seed: " + seed);
+    Log.getLog().info("Running with seed: " + seed);
 
     // Set cluster parameters
     long timeWindow = 1000000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestGreedyReservationAgent.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/reservation/planning/TestGreedyReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestGreedyReservationAgent.java
index b8a618b..ec11ffe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestGreedyReservationAgent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestGreedyReservationAgent.java
@@ -60,7 +60,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 @RunWith(Parameterized.class)
 public class TestGreedyReservationAgent {
@@ -89,7 +89,7 @@ public class TestGreedyReservationAgent {
 
     long seed = rand.nextLong();
     rand.setSeed(seed);
-    Log.info("Running with seed: " + seed);
+    Log.getLog().info("Running with seed: " + seed);
 
     // setting completely loose quotas
     long timeWindow = 1000000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 082d043..6af2110 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
@@ -42,6 +42,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
@@ -144,7 +145,8 @@ public class TestRMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("info").accept("application/xml").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyClusterInfoXML(xml);
   }
@@ -204,7 +206,8 @@ public class TestRMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterInfo(json);
   }
@@ -216,7 +219,8 @@ public class TestRMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster/")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterInfo(json);
   }
@@ -228,7 +232,8 @@ public class TestRMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterInfo(json);
   }
@@ -240,7 +245,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("info").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterInfo(json);
   }
@@ -253,7 +259,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("info/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterInfo(json);
   }
@@ -264,7 +271,8 @@ public class TestRMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("info").get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterInfo(json);
   }
@@ -352,7 +360,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("metrics").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterMetricsJSON(json);
   }
@@ -364,7 +373,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("metrics/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterMetricsJSON(json);
   }
@@ -375,7 +385,8 @@ public class TestRMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("metrics").get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterMetricsJSON(json);
   }
@@ -385,7 +396,8 @@ public class TestRMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("metrics").accept("application/xml").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyClusterMetricsXML(xml);
   }
@@ -504,7 +516,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("scheduler").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterSchedulerFifo(json);
   }
@@ -516,7 +529,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("scheduler/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterSchedulerFifo(json);
   }
@@ -527,7 +541,8 @@ public class TestRMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler").get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterSchedulerFifo(json);
   }
@@ -539,7 +554,8 @@ public class TestRMWebServices extends JerseyTestBase {
         .path("scheduler").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifySchedulerFifoXML(xml);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 7d439f1..da28754 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
@@ -31,6 +31,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -156,7 +157,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -184,7 +186,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -204,7 +207,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path(path).accept(media).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -227,7 +231,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .path("apps")
         .queryParam("state", YarnApplicationState.ACCEPTED.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -254,7 +259,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParams(params)
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -271,7 +277,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParams(params)
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -303,7 +310,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParams(params)
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -320,7 +328,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParams(params)
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -348,7 +357,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .path("apps")
         .queryParam("states", YarnApplicationState.RUNNING.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("apps is not empty",
@@ -368,7 +378,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .path("apps")
         .queryParam("state", YarnApplicationState.RUNNING.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("apps is not empty",
@@ -392,7 +403,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -429,7 +441,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -461,7 +474,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("finalStatus", FinalApplicationStatus.UNDEFINED.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     System.out.println(json.toString());
@@ -484,7 +498,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("finalStatus", FinalApplicationStatus.KILLED.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("apps is not null",
@@ -508,7 +523,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -547,7 +563,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .queryParam("user",
             UserGroupInformation.getCurrentUser().getShortUserName())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     assertEquals("incorrect number of elements", 1, json.length());
@@ -571,7 +588,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("queue", "default")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -592,7 +610,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("limit", "2")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -615,7 +634,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("startedTimeBegin", String.valueOf(start))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -638,7 +658,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("startedTimeBegin", String.valueOf(start))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -661,7 +682,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("startedTimeEnd", String.valueOf(end))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("apps is not empty",
@@ -685,7 +707,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .path("apps").queryParam("startedTimeBegin", String.valueOf(start))
         .queryParam("startedTimeEnd", String.valueOf(end))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -717,7 +740,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("finishedTimeBegin", String.valueOf(start))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -749,7 +773,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("finishedTimeEnd", String.valueOf(end))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -784,7 +809,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .path("apps").queryParam("finishedTimeBegin", String.valueOf(start))
         .queryParam("finishedTimeEnd", String.valueOf(end))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -818,7 +844,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("applicationTypes", "MAPREDUCE")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -834,7 +861,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
             .queryParam("applicationTypes", "YARN")
             .queryParam("applicationTypes", "MAPREDUCE")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -853,7 +881,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("apps")
             .queryParam("applicationTypes", "YARN,NON-YARN")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -871,7 +900,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     response = r.path("ws").path("v1").path("cluster")
         .path("apps").queryParam("applicationTypes", "")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -885,7 +915,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
             .queryParam("applicationTypes", "YARN,NON-YARN")
             .queryParam("applicationTypes", "MAPREDUCE")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -899,7 +930,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
             .queryParam("applicationTypes", "YARN")
             .queryParam("applicationTypes", "")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -914,7 +946,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("apps")
             .queryParam("applicationTypes", ",,, ,, YARN ,, ,")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -929,7 +962,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("apps")
             .queryParam("applicationTypes", ",,, ,,  ,, ,")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -942,7 +976,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("apps")
             .queryParam("applicationTypes", "YARN, ,NON-YARN, ,,")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -962,7 +997,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
             .queryParam("applicationTypes", " YARN, ,  ,,,")
             .queryParam("applicationTypes", "MAPREDUCE , ,, ,")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     apps = json.getJSONObject("apps");
@@ -1006,7 +1042,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("cluster")
           .path("appstatistics")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject appsStatInfo = json.getJSONObject("appStatInfo");
@@ -1032,7 +1069,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           .path("appstatistics")
           .queryParam("states", YarnApplicationState.ACCEPTED.toString())
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       appsStatInfo = json.getJSONObject("appStatInfo");
@@ -1049,7 +1087,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           .path("appstatistics")
           .queryParam("applicationTypes", "MAPREDUCE")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       appsStatInfo = json.getJSONObject("appStatInfo");
@@ -1076,7 +1115,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           .queryParam("applicationTypes", "MAPREDUCE,OTHER")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject exception = json.getJSONObject("RemoteException");
@@ -1099,7 +1139,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
               + "," + YarnApplicationState.ACCEPTED.toString())
           .queryParam("applicationTypes", "MAPREDUCE")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       appsStatInfo = json.getJSONObject("appStatInfo");
@@ -1123,7 +1164,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           .path("appstatistics").queryParam("states", "wrong_state")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       exception = json.getJSONObject("RemoteException");
@@ -1215,7 +1257,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -1253,7 +1296,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
 
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -1278,7 +1322,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").path(path).accept(media).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     assertEquals("incorrect number of elements", 1, json.length());
@@ -1295,7 +1340,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").path(app1.getApplicationId().toString())
         .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
 
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -1619,7 +1665,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -1658,7 +1705,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -1696,7 +1744,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
 
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -1722,7 +1771,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("apps").path(path).path("appattempts").accept(media)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jsonAppAttempts = json.getJSONObject("appAttempts");
@@ -1753,7 +1803,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .path("apps").path(app1.getApplicationId().toString())
         .path("appattempts").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
 
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 8b7e476..e884a88 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
@@ -53,6 +53,7 @@ import javax.xml.parsers.ParserConfigurationException;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
@@ -341,9 +342,9 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
             .constructWebResource("apps", app.getApplicationId().toString(),
               "state").accept(mediaType).get(ClientResponse.class);
       assertResponseStatusCode(Status.OK, response.getStatusInfo());
-      if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+      if (mediaType.contains(MediaType.APPLICATION_JSON)) {
         verifyAppStateJson(response, RMAppState.ACCEPTED);
-      } else if (mediaType.equals(MediaType.APPLICATION_XML)) {
+      } else if (mediaType.contains(MediaType.APPLICATION_XML)) {
         verifyAppStateXML(response, RMAppState.ACCEPTED);
       }
     }
@@ -386,7 +387,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
           continue;
         }
         assertResponseStatusCode(Status.ACCEPTED, response.getStatusInfo());
-        if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
           verifyAppStateJson(response, RMAppState.FINAL_SAVING,
             RMAppState.KILLED, RMAppState.KILLING, RMAppState.ACCEPTED);
         } else {
@@ -494,7 +495,8 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   protected static void verifyAppStateJson(ClientResponse response,
       RMAppState... states) throws JSONException {
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     String responseState = json.getString("state");
@@ -511,7 +513,8 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   protected static void verifyAppStateXML(ClientResponse response,
       RMAppState... appStates) throws ParserConfigurationException,
       IOException, SAXException {
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -682,10 +685,10 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
       throws JSONException, ParserConfigurationException, IOException,
       SAXException {
     String ret = "";
-    if (resp.getType().equals(MediaType.APPLICATION_JSON_TYPE)) {
+    if (resp.getType().toString().contains(MediaType.APPLICATION_JSON)) {
       JSONObject json = resp.getEntity(JSONObject.class);
       ret = validateGetNewApplicationJsonResponse(json);
-    } else if (resp.getType().equals(MediaType.APPLICATION_XML_TYPE)) {
+    } else if (resp.getType().toString().contains(MediaType.APPLICATION_XML)) {
       String xml = resp.getEntity(String.class);
       ret = validateGetNewApplicationXMLResponse(xml);
     } else {
@@ -1040,7 +1043,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
       if(!isCapacityScheduler) {
         expectedQueue = "root." + webserviceUserName;
       }
-      if (contentType.equals(MediaType.APPLICATION_JSON)) {
+      if (contentType.contains(MediaType.APPLICATION_JSON)) {
         verifyAppQueueJson(response, expectedQueue);
       } else {
         verifyAppQueueXML(response, expectedQueue);
@@ -1105,7 +1108,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
           continue;
         }
         assertResponseStatusCode(Status.OK, response.getStatusInfo());
-        if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
           verifyAppPriorityJson(response, modifiedPriority);
         } else {
           verifyAppPriorityXML(response, modifiedPriority);
@@ -1116,7 +1119,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
                 "priority")
             .accept(mediaType).get(ClientResponse.class);
         assertResponseStatusCode(Status.OK, response.getStatusInfo());
-        if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
           verifyAppPriorityJson(response, modifiedPriority);
         } else {
           verifyAppPriorityXML(response, modifiedPriority);
@@ -1189,7 +1192,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
         if(!isCapacityScheduler) {
           expectedQueue = "root.test";
         }
-        if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
           verifyAppQueueJson(response, expectedQueue);
         } else {
           verifyAppQueueXML(response, expectedQueue);
@@ -1233,9 +1236,11 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     jm.marshallToJSON(targetQueue, sw);
     return sw.toString();
   }
- protected static void verifyAppPriorityJson(ClientResponse response,
+
+  protected static void verifyAppPriorityJson(ClientResponse response,
       int expectedPriority) throws JSONException {
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     int responsePriority = json.getInt("priority");
@@ -1245,7 +1250,8 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   protected static void verifyAppPriorityXML(ClientResponse response,
       int expectedPriority)
           throws ParserConfigurationException, IOException, SAXException {
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -1263,7 +1269,8 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
       verifyAppQueueJson(ClientResponse response, String queue)
           throws JSONException {
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     String responseQueue = json.getString("queue");
@@ -1273,7 +1280,8 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   protected static void
       verifyAppQueueXML(ClientResponse response, String queue)
           throws ParserConfigurationException, IOException, SAXException {
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 bbdfdd8..1108f1a 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
@@ -29,6 +29,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -175,7 +176,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterScheduler(json);
   }
@@ -186,7 +188,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterScheduler(json);
   }
@@ -196,7 +199,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterScheduler(json);
   }
@@ -207,7 +211,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler/").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -482,7 +487,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
       WebResource r = resource();
       ClientResponse response = r.path("ws/v1/cluster/scheduler")
         .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilder db = DocumentBuilderFactory.newInstance()
         .newDocumentBuilder();
@@ -563,7 +569,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("cluster")
           .path("scheduler/").accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       JSONObject schedulerInfo = json.getJSONObject("scheduler").getJSONObject(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 fe01d28..095b076 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
@@ -689,7 +689,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
       getDelegationTokenFromResponse(ClientResponse response)
           throws IOException, ParserConfigurationException, SAXException,
           JSONException {
-    if (response.getType().toString().equals(MediaType.APPLICATION_JSON)) {
+    if (response.getType().toString().contains(MediaType.APPLICATION_JSON)) {
       return getDelegationTokenFromJson(response.getEntity(JSONObject.class));
     }
     return getDelegationTokenFromXML(response.getEntity(String.class));
@@ -760,7 +760,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
 
   private static String generateRenewTokenBody(String mediaType, String token) {
     String body = "";
-    if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+    if (mediaType.contains(MediaType.APPLICATION_JSON)) {
       body = "{\"token\": \"" + token + "\" }";
     } else {
       body =


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


[41/50] [abbrv] hadoop git commit: YARN-2306. Add test for leakage of reservation metrics in fair scheduler. (Hong Zhiguo and Yufei Gu via subru).

Posted by vi...@apache.org.
YARN-2306. Add test for leakage of reservation metrics in fair scheduler. (Hong Zhiguo and Yufei Gu via subru).


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

Branch: refs/heads/HDFS-9806
Commit: b2c4f24c31e73faa8f71d44db5de3aa91e3b7d5e
Parents: 28660f5
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Oct 27 17:43:13 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Thu Oct 27 17:43:13 2016 -0700

----------------------------------------------------------------------
 .../scheduler/fair/TestFairScheduler.java       | 52 ++++++++++++++++++++
 1 file changed, 52 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c4f24c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index e28b35a..f17726c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -4761,4 +4761,56 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     rm1.stop();
     rm2.stop();
   }
+
+  @Test
+  public void testReservationMetrics() throws IOException {
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+    QueueMetrics metrics = scheduler.getRootQueueMetrics();
+
+    RMNode node1 =
+        MockNodes
+            .newNodeInfo(1, Resources.createResource(4096, 4), 1, "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent);
+
+    ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
+    createApplicationWithAMResource(appAttemptId, "default", "user1", null);
+
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+
+    createSchedulingRequestExistingApplication(1024, 1, 1, appAttemptId);
+    scheduler.handle(updateEvent);
+
+    // no reservation yet
+    assertEquals(0, metrics.getReservedContainers());
+    assertEquals(0, metrics.getReservedMB());
+    assertEquals(0, metrics.getReservedVirtualCores());
+
+    // create reservation of {4096, 4}
+    createSchedulingRequestExistingApplication(4096, 4, 1, appAttemptId);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+
+    // reservation created
+    assertEquals(1, metrics.getReservedContainers());
+    assertEquals(4096, metrics.getReservedMB());
+    assertEquals(4, metrics.getReservedVirtualCores());
+
+    // remove AppAttempt
+    AppAttemptRemovedSchedulerEvent attRemoveEvent =
+        new AppAttemptRemovedSchedulerEvent(
+            appAttemptId,
+            RMAppAttemptState.KILLED,
+            false);
+    scheduler.handle(attRemoveEvent);
+
+    // The reservation metrics should be subtracted
+    assertEquals(0, metrics.getReservedContainers());
+    assertEquals(0, metrics.getReservedMB());
+    assertEquals(0, metrics.getReservedVirtualCores());
+  }
 }


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


[50/50] [abbrv] hadoop git commit: YARN-4963. capacity scheduler: Make number of OFF_SWITCH assignments per heartbeat configurable. Contributed by Nathan Roberts

Posted by vi...@apache.org.
YARN-4963. capacity scheduler: Make number of OFF_SWITCH assignments per heartbeat configurable. Contributed by Nathan Roberts


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

Branch: refs/heads/HDFS-9806
Commit: 1eae719bcead45915977aa220324650eab3c1b9e
Parents: 1b79c41
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Oct 28 17:30:15 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Oct 28 17:30:15 2016 +0000

----------------------------------------------------------------------
 .../conf/capacity-scheduler.xml                 | 12 ++++++++
 .../CapacitySchedulerConfiguration.java         | 21 +++++++++++++
 .../scheduler/capacity/ParentQueue.java         | 30 +++++++++++++-----
 .../scheduler/capacity/TestParentQueue.java     | 32 +++++++++++++-------
 4 files changed, 77 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1eae719b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
index 30f4eb9..6ac726e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
@@ -131,4 +131,16 @@
     </description>
   </property>
 
+  <property>
+    <name>yarn.scheduler.capacity.per-node-heartbeat.maximum-offswitch-assignments</name>
+    <value>1</value>
+    <description>
+      Controls the number of OFF_SWITCH assignments allowed
+      during a node's heartbeat. Increasing this value can improve
+      scheduling rate for OFF_SWITCH containers. Lower values reduce
+      "clumping" of applications on particular nodes. The default is 1.
+      Legal values are 1-MAX_INT. This config is refreshable.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1eae719b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index d5d1374..6db5074 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -191,6 +191,13 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
       PREFIX + "rack-locality-full-reset";
 
   @Private
+  public static final int DEFAULT_OFFSWITCH_PER_HEARTBEAT_LIMIT = 1;
+
+  @Private
+  public static final String OFFSWITCH_PER_HEARTBEAT_LIMIT =
+      PREFIX + "per-node-heartbeat.maximum-offswitch-assignments";
+
+  @Private
   public static final boolean DEFAULT_RACK_LOCALITY_FULL_RESET = true;
 
   @Private
@@ -713,6 +720,20 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     return getBoolean(ENABLE_USER_METRICS, DEFAULT_ENABLE_USER_METRICS);
   }
 
+  public int getOffSwitchPerHeartbeatLimit() {
+    int limit = getInt(OFFSWITCH_PER_HEARTBEAT_LIMIT,
+        DEFAULT_OFFSWITCH_PER_HEARTBEAT_LIMIT);
+    if (limit < 1) {
+      LOG.warn(OFFSWITCH_PER_HEARTBEAT_LIMIT + "(" + limit + ") < 1. Using 1.");
+      limit = 1;
+    }
+    return limit;
+  }
+
+  public void setOffSwitchPerHeartbeatLimit(int limit) {
+    setInt(OFFSWITCH_PER_HEARTBEAT_LIMIT, limit);
+  }
+
   public int getNodeLocalityDelay() {
     return getInt(NODE_LOCALITY_DELAY, DEFAULT_NODE_LOCALITY_DELAY);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1eae719b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index ffb6892..a69af6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -76,6 +76,7 @@ public class ParentQueue extends AbstractCSQueue {
   volatile int numApplications;
   private final CapacitySchedulerContext scheduler;
   private boolean needToResortQueuesAtNextAllocation = false;
+  private int offswitchPerHeartbeatLimit;
 
   private final RecordFactory recordFactory = 
     RecordFactoryProvider.getRecordFactory(null);
@@ -125,12 +126,16 @@ public class ParentQueue extends AbstractCSQueue {
         }
       }
 
+      offswitchPerHeartbeatLimit =
+        csContext.getConfiguration().getOffSwitchPerHeartbeatLimit();
+
       LOG.info(queueName + ", capacity=" + this.queueCapacities.getCapacity()
           + ", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity()
           + ", maxCapacity=" + this.queueCapacities.getMaximumCapacity()
           + ", absoluteMaxCapacity=" + this.queueCapacities
           .getAbsoluteMaximumCapacity() + ", state=" + state + ", acls="
           + aclsString + ", labels=" + labelStrBuilder.toString() + "\n"
+          + ", offswitchPerHeartbeatLimit = " + getOffSwitchPerHeartbeatLimit()
           + ", reservationsContinueLooking=" + reservationsContinueLooking);
     } finally {
       writeLock.unlock();
@@ -210,6 +215,11 @@ public class ParentQueue extends AbstractCSQueue {
 
   }
 
+  @Private
+  public int getOffSwitchPerHeartbeatLimit() {
+    return offswitchPerHeartbeatLimit;
+  }
+
   private QueueUserACLInfo getUserAclInfo(
       UserGroupInformation user) {
     try {
@@ -427,6 +437,7 @@ public class ParentQueue extends AbstractCSQueue {
   public CSAssignment assignContainers(Resource clusterResource,
       FiCaSchedulerNode node, ResourceLimits resourceLimits,
       SchedulingMode schedulingMode) {
+    int offswitchCount = 0;
     try {
       writeLock.lock();
       // if our queue cannot access this node, just return
@@ -582,13 +593,18 @@ public class ParentQueue extends AbstractCSQueue {
                   + getAbsoluteUsedCapacity());
         }
 
-        // Do not assign more than one container if this isn't the root queue
-        // or if we've already assigned an off-switch container
-        if (!rootQueue || assignment.getType() == NodeType.OFF_SWITCH) {
+        if (assignment.getType() == NodeType.OFF_SWITCH) {
+          offswitchCount++;
+        }
+
+        // Do not assign more containers if this isn't the root queue
+        // or if we've already assigned enough OFF_SWITCH containers in
+        // this pass
+        if (!rootQueue || offswitchCount >= getOffSwitchPerHeartbeatLimit()) {
           if (LOG.isDebugEnabled()) {
-            if (rootQueue && assignment.getType() == NodeType.OFF_SWITCH) {
-              LOG.debug("Not assigning more than one off-switch container,"
-                  + " assignments so far: " + assignment);
+            if (rootQueue && offswitchCount >= getOffSwitchPerHeartbeatLimit()) {
+              LOG.debug("Assigned maximum number of off-switch containers: " +
+                  offswitchCount + ", assignments so far: " + assignment);
             }
           }
           break;
@@ -1046,4 +1062,4 @@ public class ParentQueue extends AbstractCSQueue {
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1eae719b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index 890e998..42a8872 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -29,6 +29,7 @@ import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.when;
 
 import java.util.HashMap;
@@ -613,6 +614,8 @@ public class TestParentQueue {
   public void testOffSwitchScheduling() throws Exception {
     // Setup queue configs
     setupSingleLevelQueues(csConf);
+    csConf.setOffSwitchPerHeartbeatLimit(2);
+
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
@@ -643,12 +646,18 @@ public class TestParentQueue {
     queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
     .incPending(Resources.createResource(1 * GB));
     
-    // Simulate B returning a container on node_0
-    stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
+    // Simulate returning 2 containers on node_0 before offswitch limit
+    stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
+
     root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
-    verifyQueueMetrics(a, 0*GB, clusterResource);
+    InOrder allocationOrder = inOrder(a, b);
+    allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource),
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
+    allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource),
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
+    verifyQueueMetrics(a, 1*GB, clusterResource);
     verifyQueueMetrics(b, 1*GB, clusterResource);
     
     // Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G
@@ -657,27 +666,28 @@ public class TestParentQueue {
     stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_1, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
-    InOrder allocationOrder = inOrder(a, b);
-    allocationOrder.verify(a).assignContainers(eq(clusterResource), 
+    allocationOrder = inOrder(a, b);
+    allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource),
         any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
-    allocationOrder.verify(b).assignContainers(eq(clusterResource), 
+    allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource),
         any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
-    verifyQueueMetrics(a, 2*GB, clusterResource);
+    verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
     
     // Now, B should get the scheduling opportunity 
     // since A has 2/6G while B has 2/14G, 
-    // However, since B returns off-switch, A won't get an opportunity
+    // A also gets an opportunity because offswitchlimit not reached
     stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
     stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH);
+
     root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(b, a);
-    allocationOrder.verify(b).assignContainers(eq(clusterResource), 
+    allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource),
         any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
-    allocationOrder.verify(a).assignContainers(eq(clusterResource), 
+    allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource),
         any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
-    verifyQueueMetrics(a, 2*GB, clusterResource);
+    verifyQueueMetrics(a, 4*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
 
   }


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


[27/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 cbe360a..e77785b 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
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 
 import javax.ws.rs.core.MediaType;
 
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -91,7 +92,8 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterScheduler(json);
   }
@@ -102,7 +104,8 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyClusterScheduler(json);
   }
@@ -120,7 +123,8 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     JSONArray subQueueInfo = json.getJSONObject("scheduler")
         .getJSONObject("schedulerInfo").getJSONObject("rootQueue")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 29a38d9..c286186 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
@@ -29,6 +29,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -194,7 +195,8 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
     ClientResponse response =
         r.path("ws").path("v1").path("cluster").path("scheduler")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifySchedulerInfoJson(json);
   }
@@ -205,7 +207,8 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
     ClientResponse response =
         r.path("ws").path("v1").path("cluster").path("scheduler/")
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifySchedulerInfoJson(json);
 
@@ -216,7 +219,8 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("scheduler").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifySchedulerInfoJson(json);
   }
@@ -227,7 +231,8 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
     ClientResponse response =
         r.path("ws").path("v1").path("cluster").path("scheduler")
             .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 0cfaac1..5e6fd4e 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
@@ -30,6 +30,7 @@ import javax.ws.rs.core.MediaType;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -137,7 +138,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals(1, nlsifo.getNodeLabels().size());
     for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) {
@@ -160,7 +162,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals(2, nlsifo.getNodeLabels().size());
     // Verify exclusivity for 'y' as false
@@ -214,7 +217,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("label-mappings").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     LabelsToNodesInfo ltni = response.getEntity(LabelsToNodesInfo.class);
     assertEquals(2, ltni.getLabelsToNodes().size());
     NodeIDsInfo nodes = ltni.getLabelsToNodes().get(
@@ -233,7 +237,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
             .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     ltni = response.getEntity(LabelsToNodesInfo.class);
     assertEquals(1, ltni.getLabelsToNodes().size());
     nodes = ltni.getLabelsToNodes().get(new NodeLabelInfo("a"));
@@ -245,7 +250,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("nodes").path("nid:0")
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertTrue(nlsifo.getNodeLabelsInfo().contains(new NodeLabelInfo("a")));
 
@@ -269,7 +275,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("nodes").path("nid:0")
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertTrue(nlsifo.getNodeLabelsInfo().contains(
         new NodeLabelInfo("b", false)));
@@ -294,7 +301,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-to-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     NodeToLabelsInfo ntlinfo = response.getEntity(NodeToLabelsInfo.class);
     NodeLabelsInfo nlinfo = ntlinfo.getNodeToLabels().get("nid:0");
     assertEquals(1, nlinfo.getNodeLabels().size());
@@ -318,7 +326,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("nodes").path("nid:0")
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertTrue(nlsifo.getNodeLabelsInfo().isEmpty());
     
@@ -341,7 +350,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("nodes").path("nid:0")
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertTrue(nlsifo.getNodeLabelsInfo().contains(new NodeLabelInfo("a")));
     
@@ -362,7 +372,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("nodes").path("nid:0")
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertTrue(nlsifo.getNodeLabelsInfo().contains(new NodeLabelInfo("a")));
     
@@ -379,7 +390,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals(2, nlsifo.getNodeLabels().size());
     
@@ -398,7 +410,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals(1, nlsifo.getNodeLabels().size());
     for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) {
@@ -421,7 +434,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals(0, nlsifo.getNodeLabels().size());
 
@@ -475,7 +489,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("get-node-to-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     ntlinfo = response.getEntity(NodeToLabelsInfo.class);
     nlinfo = ntlinfo.getNodeToLabels().get("nid:0");
     assertEquals(1, nlinfo.getNodeLabels().size());
@@ -496,7 +511,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("get-node-to-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     ntlinfo = response.getEntity(NodeToLabelsInfo.class);
     nlinfo = ntlinfo.getNodeToLabels().get("nid:0");
     assertEquals(1, nlinfo.getNodeLabels().size());
@@ -518,7 +534,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals(new NodeLabelInfo("y", false),
         nlsifo.getNodeLabelsInfo().get(0));
@@ -541,7 +558,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertTrue(nlsifo.getNodeLabelsInfo().isEmpty());
 
@@ -563,7 +581,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     nlsifo = response.getEntity(NodeLabelsInfo.class);
     assertEquals("z", nlsifo.getNodeLabelsInfo().get(0).getName());
     assertFalse(nlsifo.getNodeLabelsInfo().get(0).getExclusivity());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 718091f..71f8950 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
@@ -32,6 +32,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -154,7 +155,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster").path("nodes")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject nodes = json.getJSONObject("nodes");
@@ -201,7 +203,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .path("nodes").queryParam("states", NodeState.NEW.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject nodes = json.getJSONObject("nodes");
@@ -223,7 +226,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .path("nodes")
         .queryParam("states", NodeState.DECOMMISSIONED.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("nodes is not empty",
@@ -246,7 +250,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
 
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -279,7 +284,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .path("nodes").queryParam("states", NodeState.LOST.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     JSONObject nodes = json.getJSONObject("nodes");
     assertEquals("incorrect number of elements", 1, nodes.length());
@@ -310,7 +316,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .path("nodes").path("h2:1234").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     JSONObject info = json.getJSONObject("node");
     String id = info.get("id").toString();
@@ -336,7 +343,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").queryParam("states", "running")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject nodes = json.getJSONObject("nodes");
@@ -354,7 +362,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").queryParam("states", "UNHEALTHY")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("nodes is not empty",
@@ -369,7 +378,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path(path).accept(media).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject nodes = json.getJSONObject("nodes");
@@ -415,7 +425,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").path(nodeid).accept(media).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject info = json.getJSONObject("node");
@@ -438,7 +449,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -464,7 +476,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -490,7 +503,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String msg = response.getEntity(String.class);
       System.out.println(msg);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -533,7 +547,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -557,7 +572,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -581,7 +597,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .path("nodes").path("h1:1234").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
 
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -602,7 +619,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
 
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -631,7 +649,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .queryParam("states", Joiner.on(',').join(EnumSet.allOf(NodeState.class)))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     JSONObject nodes = json.getJSONObject("nodes");
     assertEquals("incorrect number of elements", 1, nodes.length());
@@ -663,7 +682,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         .path("nodes").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject nodes = json.getJSONObject("nodes");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 41863f6..657bec4 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
@@ -39,6 +39,7 @@ import javax.ws.rs.core.MediaType;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -968,7 +969,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     }
 
     System.out.println("RESPONSE:" + response);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     assertEquals("incorrect number of elements", 1, json.length());
@@ -1055,7 +1057,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     }
 
     System.out.println("RESPONSE:" + response);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertResponseStatusCode(Status.OK, response.getStatusInfo());
 
   }
@@ -1104,7 +1107,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     }
 
     System.out.println("RESPONSE:" + response);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertResponseStatusCode(Status.OK, response.getStatusInfo());
   }
 
@@ -1134,7 +1138,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
       return null;
     }
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertResponseStatusCode(status, response.getStatusInfo());
 
     return response.getEntity(JSONObject.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.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/TestRMWebServicesSchedulerActivities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java
index d7b0581..a0bd951 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java
@@ -23,6 +23,7 @@ import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.core.util.MultivaluedMapImpl;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -79,7 +80,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       nm.nodeHeartbeat(true);
@@ -89,7 +91,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 11);
@@ -139,7 +142,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       nm.nodeHeartbeat(true);
@@ -149,7 +153,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 0);
@@ -172,7 +177,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       Thread.sleep(1000);
@@ -181,7 +187,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 0);
@@ -218,7 +225,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       nm.nodeHeartbeat(true);
@@ -228,7 +236,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 0);
@@ -269,7 +278,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       nm2.nodeHeartbeat(true);
@@ -278,7 +288,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 1);
@@ -295,7 +306,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       nm2.nodeHeartbeat(true);
@@ -304,7 +316,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 1);
@@ -329,7 +342,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       nm2.nodeHeartbeat(true);
@@ -338,7 +352,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 1);
@@ -373,7 +388,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       nm.nodeHeartbeat(true);
@@ -383,7 +399,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 1);
@@ -519,7 +536,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       nm.nodeHeartbeat(true);
       Thread.sleep(5000);
@@ -528,7 +546,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 1);
@@ -571,7 +590,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       nm.nodeHeartbeat(true);
       Thread.sleep(5000);
@@ -580,7 +600,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 10);
@@ -623,7 +644,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       nm.nodeHeartbeat(true);
       Thread.sleep(5000);
@@ -632,7 +654,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 0);
@@ -657,14 +680,16 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       //Get JSON
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 0);
@@ -705,7 +730,8 @@ public class TestRMWebServicesSchedulerActivities
       ClientResponse response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
 
       nm2.nodeHeartbeat(true);
@@ -714,7 +740,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 1);
@@ -726,7 +753,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       nm2.nodeHeartbeat(true);
@@ -735,7 +763,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 2);
@@ -755,7 +784,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       nm2.nodeHeartbeat(true);
@@ -764,7 +794,8 @@ public class TestRMWebServicesSchedulerActivities
       response = r.path("ws").path("v1").path("cluster").path(
           "scheduler/app-activities").queryParams(params).accept(
           MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       json = response.getEntity(JSONObject.class);
 
       verifyNumberOfAllocations(json, 3);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 a15d656..2925e84 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
@@ -26,6 +26,7 @@ import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.http.RestCsrfPreventionFilter;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.VersionInfo;
@@ -120,7 +121,8 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
         .get(ClientResponse.class);
     assertTrue("Should have been accepted", response.getStatus() ==
                                             Status.OK.getStatusCode());
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyClusterInfoXML(xml);
   }
@@ -144,7 +146,8 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
         .get(ClientResponse.class);
     assertTrue("Should have been accepted", response.getStatus() ==
                                             Status.OK.getStatusCode());
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyClusterInfoXML(xml);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
index 1b8b7f9..7119c4d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
@@ -55,7 +55,7 @@
           <artifactId>jasper-compiler</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>jsp-2.1-jetty</artifactId>
         </exclusion>
       </exclusions>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
index eca244e..c4786c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
@@ -73,6 +73,10 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-auth</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
 
@@ -425,6 +429,18 @@
         </exclusion>
       </exclusions>
     </dependency>
+
+    <!-- Jetty 9 is needed by the ATS code -->
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-webapp</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index b185448..2ed5d96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -369,7 +369,19 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "org.apache.hadoop.yarn.server.timelineservice.storage." +
               "HBaseTimelineReaderImpl");
       config.setInt("hfile.format.version", 3);
-      server = new TimelineReaderServer();
+      server = new TimelineReaderServer() {
+        @Override
+        protected void setupOptions(Configuration conf) {
+          // The parent code tries to use HttpServer2 from this version of
+          // Hadoop, but the tests are loading in HttpServer2 from
+          // ${hbase-compatible-hadoop.version}.  This version uses Jetty 9
+          // while ${hbase-compatible-hadoop.version} uses Jetty 6, and there
+          // are many differences, including classnames and packages.
+          // We do nothing here, so that we don't cause a NoSuchMethodError.
+          // Once ${hbase-compatible-hadoop.version} is changed to Hadoop 3,
+          // we should be able to remove this @Override.
+        }
+      };
       server.init(config);
       server.start();
       serverPort = server.getWebServerPort();
@@ -478,7 +490,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "1002345678919");
       ClientResponse resp = getResponse(client, uri);
       FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entity);
       assertEquals("user1@flow_name/1002345678919", entity.getId());
       assertEquals(3, entity.getMetrics().size());
@@ -523,7 +536,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       ClientResponse resp = getResponse(client, uri);
       Set<FlowRunEntity> entities =
           resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       for (FlowRunEntity entity : entities) {
@@ -541,7 +555,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (FlowRunEntity entity : entities) {
@@ -557,7 +572,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "createdtimestart=1425016501030");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (FlowRunEntity entity : entities) {
@@ -573,7 +589,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "createdtimestart=1425016500999&createdtimeend=1425016501035");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       for (FlowRunEntity entity : entities) {
@@ -592,7 +609,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "createdtimeend=1425016501030");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (FlowRunEntity entity : entities) {
@@ -608,7 +626,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "fields=metrics");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       for (FlowRunEntity entity : entities) {
@@ -644,7 +663,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       ClientResponse resp = getResponse(client, uri);
       Set<FlowRunEntity> entities =
           resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       int metricCnt = 0;
@@ -662,7 +682,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "metricstoretrieve=!(MAP_,HDFS_)");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       metricCnt = 0;
@@ -2101,7 +2122,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       ClientResponse resp = getResponse(client, uri);
       Set<FlowActivityEntity> entities =
           resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(0, entities.size());
     } finally {
@@ -2131,7 +2153,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(0, entities.size());
     } finally {
@@ -2148,7 +2171,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; charset=utf-8",
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(0, entities.size());
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
index a6b4cee..523ae82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
@@ -83,7 +83,7 @@
 
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
 
     <dependency>
@@ -139,6 +139,10 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-mapreduce-client-core</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
 
@@ -173,6 +177,18 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-mapreduce-client-core</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty-sslengine</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
index 29ef1f8..f36c636 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
@@ -41,6 +41,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity;
@@ -111,7 +112,8 @@ public class TimelineCollectorWebService {
    * @return description of timeline web service.
    */
   @GET
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public AboutInfo about(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index 110d1dc..9cbcc94 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -119,14 +119,7 @@ public class TimelineReaderServer extends CompositeService {
             .addEndpoint(URI.create("http://" + bindAddress));
       readerWebServer = builder.build();
 
-      Map<String, String> options = new HashMap<>();
-      String username = conf.get(HADOOP_HTTP_STATIC_USER,
-          DEFAULT_HADOOP_HTTP_STATIC_USER);
-      options.put(HADOOP_HTTP_STATIC_USER, username);
-      HttpServer2.defineFilter(readerWebServer.getWebAppContext(),
-          "static_user_filter_timeline",
-          StaticUserWebFilter.StaticUserFilter.class.getName(),
-          options, new String[] {"/*"});
+      setupOptions(conf);
 
       readerWebServer.addJerseyResourcePackage(
           TimelineReaderWebServices.class.getPackage().getName() + ";"
@@ -143,6 +136,22 @@ public class TimelineReaderServer extends CompositeService {
     }
   }
 
+  /**
+   * Sets up some options and filters.
+   *
+   * @param conf Configuration
+   */
+  protected void setupOptions(Configuration conf) {
+    Map<String, String> options = new HashMap<>();
+    String username = conf.get(HADOOP_HTTP_STATIC_USER,
+        DEFAULT_HADOOP_HTTP_STATIC_USER);
+    options.put(HADOOP_HTTP_STATIC_USER, username);
+    HttpServer2.defineFilter(readerWebServer.getWebAppContext(),
+        "static_user_filter_timeline",
+        StaticUserWebFilter.StaticUserFilter.class.getName(),
+        options, new String[] {"/*"});
+  }
+
   @VisibleForTesting
   int getWebServerPort() {
     return readerWebServer.getConnectorAddress(0).getPort();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
index db0c4e1..139a1be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
@@ -44,6 +44,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
@@ -199,7 +200,7 @@ public class TimelineReaderWebServices {
    * @return information about the cluster including timeline version.
    */
   @GET
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineAbout about(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res) {
@@ -276,7 +277,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/app-uid/{uid}/entities/{entitytype}")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -414,7 +415,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/apps/{appid}/entities/{entitytype}")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -524,7 +525,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/clusters/{clusterid}/apps/{appid}/entities/{entitytype}")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -621,7 +622,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/entity-uid/{uid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getEntity(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -716,7 +717,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/apps/{appid}/entities/{entitytype}/{entityid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getEntity(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -787,7 +788,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/clusters/{clusterid}/apps/{appid}/entities/{entitytype}/{entityid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getEntity(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -859,7 +860,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/run-uid/{uid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -927,7 +928,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/users/{userid}/flows/{flowname}/runs/{flowrunid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -968,7 +969,7 @@ public class TimelineReaderWebServices {
   @GET
   @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/"
       + "runs/{flowrunid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1050,7 +1051,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/flow-uid/{uid}/runs/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1135,7 +1136,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/users/{userid}/flows/{flowname}/runs/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1190,7 +1191,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/runs/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1271,7 +1272,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/flows/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlows(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1317,7 +1318,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/clusters/{clusterid}/flows/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlows(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1401,7 +1402,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/app-uid/{uid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getApp(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1492,7 +1493,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/apps/{appid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getApp(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1556,7 +1557,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/clusters/{clusterid}/apps/{appid}/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public TimelineEntity getApp(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1671,7 +1672,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/run-uid/{uid}/apps")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1798,7 +1799,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/users/{userid}/flows/{flowname}/runs/{flowrunid}/apps/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1899,7 +1900,7 @@ public class TimelineReaderWebServices {
   @GET
   @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/runs/"
       + "{flowrunid}/apps/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -1997,7 +1998,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/users/{userid}/flows/{flowname}/apps/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -2094,7 +2095,7 @@ public class TimelineReaderWebServices {
    */
   @GET
   @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/apps/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Set<TimelineEntity> getFlowApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,


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


[26/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
index a2a9c28..4bd37f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
@@ -35,6 +35,7 @@ import javax.ws.rs.core.MediaType;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
@@ -184,7 +185,8 @@ public class TestTimelineReaderWebServices {
           "timeline/clusters/cluster1/apps/app1/entities/app/id_1");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
@@ -207,7 +209,8 @@ public class TestTimelineReaderWebServices {
           "userid=user1&flowname=flow1&flowrunid=1");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
@@ -227,7 +230,8 @@ public class TestTimelineReaderWebServices {
           "fields=CONFIGS,Metrics,info");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
@@ -253,7 +257,8 @@ public class TestTimelineReaderWebServices {
           "fields=ALL");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
@@ -289,7 +294,8 @@ public class TestTimelineReaderWebServices {
           "timeline/apps/app1/entities/app/id_1");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
@@ -299,7 +305,8 @@ public class TestTimelineReaderWebServices {
       resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(4, entities.size());
     } finally {
@@ -316,7 +323,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(4, entities.size());
       assertTrue("Entities id_1, id_2, id_3 and id_4 should have been" +
@@ -339,7 +347,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       // Entities returned are based on most recent created time.
@@ -352,7 +361,8 @@ public class TestTimelineReaderWebServices {
           "clusters/cluster1/apps/app1/entities/app?limit=3");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       // Even though 2 entities out of 4 have same created time, one entity
       // is left out due to limit
@@ -372,7 +382,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_4 should have been present in response.",
@@ -383,7 +394,8 @@ public class TestTimelineReaderWebServices {
           "=1425016502010");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(3, entities.size());
       assertFalse("Entity with id_4 should not have been present in response.",
@@ -394,7 +406,8 @@ public class TestTimelineReaderWebServices {
           "1425016502010");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_4 should have been present in response.",
@@ -414,7 +427,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_1 should have been present in response.",
@@ -425,7 +439,8 @@ public class TestTimelineReaderWebServices {
           "type1:tid1_2,type2:tid2_1%60");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_1 should have been present in response.",
@@ -436,7 +451,8 @@ public class TestTimelineReaderWebServices {
           "type1:tid1_1:tid1_2,type2:tid2_1%60");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_1 should have been present in response.",
@@ -456,7 +472,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_3 should have been present in response.",
@@ -476,7 +493,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_3 should have been present in response.",
@@ -496,7 +514,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(2, entities.size());
       assertTrue("Entities with id_1 and id_2 should have been present" +
@@ -518,7 +537,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(1, entities.size());
       assertTrue("Entity with id_3 should have been present in response.",
@@ -541,7 +561,8 @@ public class TestTimelineReaderWebServices {
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       assertEquals(0, entities.size());
     } finally {
@@ -580,7 +601,8 @@ public class TestTimelineReaderWebServices {
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>() {
           });
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       int totalEntities = entities.size();
       assertEquals(2, totalEntities);
@@ -633,7 +655,8 @@ public class TestTimelineReaderWebServices {
       TimelineEntity entities1 =
           resp.getEntity(new GenericType<TimelineEntity>() {
           });
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities1);
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"
@@ -663,7 +686,8 @@ public class TestTimelineReaderWebServices {
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>() {
           });
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities);
       int totalEntities = entities.size();
       assertEquals(3, totalEntities);
@@ -734,7 +758,8 @@ public class TestTimelineReaderWebServices {
       TimelineEntity entities1 =
           resp.getEntity(new GenericType<TimelineEntity>() {
           });
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          resp.getType().toString());
       assertNotNull(entities1);
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
index 844d8d9..cb9759b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
@@ -35,7 +35,7 @@
   <dependencies>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>compile</scope>
     </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
@@ -87,8 +87,8 @@
       <artifactId>commons-logging</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
index 7236982..fc97387 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
@@ -55,12 +55,14 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletHolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -86,13 +88,14 @@ public class TestWebAppProxyServlet {
   @BeforeClass
   public static void start() throws Exception {
     server = new Server(0);
-    Context context = new Context();
+    ((QueuedThreadPool)server.getThreadPool()).setMaxThreads(10);
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/foo");
     server.setHandler(context);
     context.addServlet(new ServletHolder(TestServlet.class), "/bar");
-    server.getConnectors()[0].setHost("localhost");
+    ((ServerConnector)server.getConnectors()[0]).setHost("localhost");
     server.start();
-    originalPort = server.getConnectors()[0].getLocalPort();
+    originalPort = ((ServerConnector)server.getConnectors()[0]).getLocalPort();
     LOG.info("Running embedded servlet container at: http://localhost:"
         + originalPort);
     // This property needs to be set otherwise CORS Headers will be dropped
@@ -140,12 +143,12 @@ public class TestWebAppProxyServlet {
   @Test(timeout=5000)
   public void testWebAppProxyServlet() throws Exception {
     configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9090");
-    // overriding num of web server threads, see HttpServer.HTTP_MAXTHREADS 
-    configuration.setInt("hadoop.http.max.threads", 5);
+    // overriding num of web server threads, see HttpServer.HTTP_MAXTHREADS
+    configuration.setInt("hadoop.http.max.threads", 10);
     WebAppProxyServerForTest proxy = new WebAppProxyServerForTest();
     proxy.init(configuration);
     proxy.start();
-    
+
     int proxyPort = proxy.proxy.proxyServer.getConnectorAddress(0).getPort();
     AppReportFetcherForTest appReportFetcher = proxy.proxy.appReportFetcher;
 
@@ -252,7 +255,7 @@ public class TestWebAppProxyServlet {
   public void testAppReportForEmptyTrackingUrl() throws Exception {
     configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9090");
     // overriding num of web server threads, see HttpServer.HTTP_MAXTHREADS
-    configuration.setInt("hadoop.http.max.threads", 5);
+    configuration.setInt("hadoop.http.max.threads", 10);
     WebAppProxyServerForTest proxy = new WebAppProxyServerForTest();
     proxy.init(configuration);
     proxy.start();
@@ -310,7 +313,7 @@ public class TestWebAppProxyServlet {
   public void testWebAppProxyPassThroughHeaders() throws Exception {
     Configuration configuration = new Configuration();
     configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9091");
-    configuration.setInt("hadoop.http.max.threads", 5);
+    configuration.setInt("hadoop.http.max.threads", 10);
     WebAppProxyServerForTest proxy = new WebAppProxyServerForTest();
     proxy.init(configuration);
     proxy.start();
@@ -390,14 +393,14 @@ public class TestWebAppProxyServlet {
     return new String(data.toByteArray(), "UTF-8");
   }
 
-  private boolean isResponseCookiePresent(HttpURLConnection proxyConn, 
+  private boolean isResponseCookiePresent(HttpURLConnection proxyConn,
       String expectedName, String expectedValue) {
     Map<String, List<String>> headerFields = proxyConn.getHeaderFields();
     List<String> cookiesHeader = headerFields.get("Set-Cookie");
     if (cookiesHeader != null) {
       for (String cookie : cookiesHeader) {
         HttpCookie c = HttpCookie.parse(cookie).get(0);
-        if (c.getName().equals(expectedName) 
+        if (c.getName().equals(expectedName)
             && c.getValue().equals(expectedValue)) {
           return true;
         }
@@ -437,7 +440,7 @@ public class TestWebAppProxyServlet {
   }
 
   private class WebAppProxyForTest extends WebAppProxy {
-    
+
     HttpServer2 proxyServer;
     AppReportFetcherForTest appReportFetcher;
 
@@ -447,7 +450,7 @@ public class TestWebAppProxyServlet {
       String bindAddress = conf.get(YarnConfiguration.PROXY_ADDRESS);
       bindAddress = StringUtils.split(bindAddress, ':')[0];
       AccessControlList acl = new AccessControlList(
-          conf.get(YarnConfiguration.YARN_ADMIN_ACL, 
+          conf.get(YarnConfiguration.YARN_ADMIN_ACL,
           YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
       proxyServer = new HttpServer2.Builder()
           .setName("proxy")
@@ -464,11 +467,11 @@ public class TestWebAppProxyServlet {
       proxyServer.setAttribute(FETCHER_ATTRIBUTE,
           appReportFetcher );
       proxyServer.setAttribute(IS_SECURITY_ENABLED_ATTRIBUTE, Boolean.TRUE);
-      
+
       String proxy = WebAppUtils.getProxyHostAndPort(conf);
       String[] proxyParts = proxy.split(":");
       String proxyHost = proxyParts[0];
-      
+
       proxyServer.setAttribute(PROXY_HOST_ATTRIBUTE, proxyHost);
       proxyServer.start();
       LOG.info("Proxy server is started at port {}",
@@ -479,7 +482,7 @@ public class TestWebAppProxyServlet {
 
   private class AppReportFetcherForTest extends AppReportFetcher {
     int answer = 0;
-    
+
     public AppReportFetcherForTest(Configuration conf) {
       super(conf);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 860c2d7..0bdef7b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -372,8 +372,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
             <exclude>.gitignore</exclude>
             <exclude>.git/**</exclude>
             <exclude>.idea/**</exclude>
-	    <exclude>**/build/**</exclude>
+            <exclude>**/build/**</exclude>
             <exclude>**/patchprocess/**</exclude>
+            <exclude>**/*.js</exclude>
          </excludes>
        </configuration>
       </plugin>


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


[47/50] [abbrv] hadoop git commit: HADOOP-13770. Shell.checkIsBashSupported swallowed an interrupted exception. Contributed by Wei-Chiu Chuang

Posted by vi...@apache.org.
HADOOP-13770. Shell.checkIsBashSupported swallowed an interrupted exception. Contributed by Wei-Chiu Chuang


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

Branch: refs/heads/HDFS-9806
Commit: c017171da00a6cd71a2901c84a0298ce14a49e23
Parents: 3de1302
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Oct 28 14:57:50 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Oct 28 14:57:50 2016 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/util/Shell.java            | 6 ++++--
 .../org/apache/hadoop/yarn/server/nodemanager/NodeManager.java | 2 +-
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c017171d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index 78f77c2..0745057 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -734,8 +734,7 @@ public abstract class Shell {
     }
   }
 
-  public static final boolean isBashSupported = checkIsBashSupported();
-  private static boolean checkIsBashSupported() {
+  public static boolean checkIsBashSupported() throws InterruptedIOException {
     if (Shell.WINDOWS) {
       return false;
     }
@@ -746,6 +745,9 @@ public abstract class Shell {
       String[] args = {"bash", "-c", "echo 1000"};
       shexec = new ShellCommandExecutor(args);
       shexec.execute();
+    } catch (InterruptedIOException iioe) {
+      LOG.warn("Interrupted, unable to determine if bash is supported", iioe);
+      throw iioe;
     } catch (IOException ioe) {
       LOG.warn("Bash is not supported by the OS", ioe);
       supported = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c017171d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 37f67c4..5424464 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -750,7 +750,7 @@ public class NodeManager extends CompositeService
       // Failed to start if we're a Unix based system but we don't have bash.
       // Bash is necessary to launch containers under Unix-based systems.
       if (!Shell.WINDOWS) {
-        if (!Shell.isBashSupported) {
+        if (!Shell.checkIsBashSupported()) {
           String message =
               "Failing NodeManager start since we're on a "
                   + "Unix-based system but bash doesn't seem to be available.";


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


[06/50] [abbrv] hadoop git commit: HDFS-10455. Logging the username when deny the setOwner operation. Contributed by Tianyin Xiu

Posted by vi...@apache.org.
HDFS-10455. Logging the username when deny the setOwner operation. Contributed by Tianyin Xiu


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

Branch: refs/heads/HDFS-9806
Commit: ac35ee9393e0afce9fede1d2052e7bf4032312fd
Parents: 0c837db
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Oct 27 20:20:56 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Thu Oct 27 20:20:56 2016 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java    | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac35ee93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 417ce01..488c600 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -84,10 +84,12 @@ public class FSDirAttrOp {
       fsd.checkOwner(pc, iip);
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
-          throw new AccessControlException("Non-super user cannot change owner");
+          throw new AccessControlException("User " + pc.getUser()
+              + " is not a super user (non-super user cannot change owner).");
         }
         if (group != null && !pc.isMemberOfGroup(group)) {
-          throw new AccessControlException("User does not belong to " + group);
+          throw new AccessControlException(
+              "User " + pc.getUser() + " does not belong to " + group);
         }
       }
       unprotectedSetOwner(fsd, iip, username, group);


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


[29/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz
deleted file mode 100644
index 2aac85f..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/MyTestWebService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/MyTestWebService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/MyTestWebService.java
index f37b01a..74623a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/MyTestWebService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/MyTestWebService.java
@@ -27,12 +27,13 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import com.google.inject.Singleton;
+import org.apache.hadoop.http.JettyUtils;
 
 @Singleton
 @Path("/ws/v1/test")
 public class MyTestWebService {
   @GET
-  @Produces({ MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public MyInfo get() {
     return new MyInfo();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
index f772c77..1474c19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
@@ -38,7 +38,7 @@
   <dependencies>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
index fd63787..6e6e98b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
@@ -56,8 +56,8 @@ import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitialize
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
-import org.mortbay.jetty.servlet.FilterHolder;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -316,27 +316,31 @@ public class ApplicationHistoryServer extends CompositeService {
            YarnConfiguration.TIMELINE_SERVICE_UI_NAMES);
        WebAppContext webAppContext = httpServer.getWebAppContext();
 
-       for (String name : names) {
-         String webPath = conf.get(
-             YarnConfiguration.TIMELINE_SERVICE_UI_WEB_PATH_PREFIX + name);
-         String onDiskPath = conf.get(
-             YarnConfiguration.TIMELINE_SERVICE_UI_ON_DISK_PATH_PREFIX + name);
-         WebAppContext uiWebAppContext = new WebAppContext();
-         uiWebAppContext.setContextPath(webPath);
-         uiWebAppContext.setWar(onDiskPath);
-         final String[] ALL_URLS = { "/*" };
-         FilterHolder[] filterHolders =
-           webAppContext.getServletHandler().getFilters();
-         for (FilterHolder filterHolder: filterHolders) {
-           if (!"guice".equals(filterHolder.getName())) {
-             HttpServer2.defineFilter(uiWebAppContext, filterHolder.getName(),
-                 filterHolder.getClassName(), filterHolder.getInitParameters(),
-                 ALL_URLS);
-           }
-         }
-         LOG.info("Hosting " + name + " from " + onDiskPath + " at " + webPath);
-         httpServer.addContext(uiWebAppContext, true);
-       }
+      for (String name : names) {
+        String webPath = conf.get(
+            YarnConfiguration.TIMELINE_SERVICE_UI_WEB_PATH_PREFIX + name);
+        String onDiskPath = conf.get(
+            YarnConfiguration.TIMELINE_SERVICE_UI_ON_DISK_PATH_PREFIX + name);
+        WebAppContext uiWebAppContext = new WebAppContext();
+        uiWebAppContext.setContextPath(webPath);
+        if (onDiskPath.endsWith(".war")) {
+          uiWebAppContext.setWar(onDiskPath);
+        } else {
+          uiWebAppContext.setResourceBase(onDiskPath);
+        }
+        final String[] ALL_URLS = {"/*"};
+        FilterHolder[] filterHolders =
+            webAppContext.getServletHandler().getFilters();
+        for (FilterHolder filterHolder : filterHolders) {
+          if (!"guice".equals(filterHolder.getName())) {
+            HttpServer2.defineFilter(uiWebAppContext, filterHolder.getName(),
+                filterHolder.getClassName(), filterHolder.getInitParameters(),
+                ALL_URLS);
+          }
+        }
+        LOG.info("Hosting " + name + " from " + onDiskPath + " at " + webPath);
+        httpServer.addHandlerAtFront(uiWebAppContext);
+      }
        httpServer.start();
        conf.updateConnectAddr(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
         YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index 04fe7bc..ccc5ff9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -90,7 +91,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/about")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TimelineAbout about(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res) {
@@ -99,7 +101,8 @@ public class AHSWebServices extends WebServices {
   }
 
   @GET
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppsInfo get(@Context HttpServletRequest req,
       @Context HttpServletResponse res) {
     return getApps(req, res, null, Collections.<String> emptySet(), null, null,
@@ -108,7 +111,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/apps")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public AppsInfo getApps(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @QueryParam("state") String stateQuery,
@@ -131,7 +135,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public AppInfo getApp(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId) {
@@ -141,7 +146,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public AppAttemptsInfo getAppAttempts(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId) {
@@ -151,7 +157,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts/{appattemptid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public AppAttemptInfo getAppAttempt(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId,
@@ -162,7 +169,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts/{appattemptid}/containers")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public ContainersInfo getContainers(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId,
@@ -173,7 +181,8 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/apps/{appid}/appattempts/{appattemptid}/containers/{containerid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Override
   public ContainerInfo getContainer(@Context HttpServletRequest req,
       @Context HttpServletResponse res, @PathParam("appid") String appId,
@@ -206,7 +215,7 @@ public class AHSWebServices extends WebServices {
 
   @GET
   @Path("/containerlogs/{containerid}/{filename}")
-  @Produces({ MediaType.TEXT_PLAIN })
+  @Produces({ MediaType.TEXT_PLAIN + "; " + JettyUtils.UTF_8 })
   @Public
   @Unstable
   public Response getLogs(@Context HttpServletRequest req,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
index 5c80132..d4acbe4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
@@ -45,6 +45,7 @@ import javax.ws.rs.core.Response;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
@@ -86,7 +87,8 @@ public class TimelineWebServices {
    * Return the description of the timeline web services.
    */
   @GET
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelineAbout about(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res) {
@@ -99,7 +101,8 @@ public class TimelineWebServices {
    */
   @GET
   @Path("/{entityType}")
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelineEntities getEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -142,7 +145,8 @@ public class TimelineWebServices {
    */
   @GET
   @Path("/{entityType}/{entityId}")
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelineEntity getEntity(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -177,7 +181,8 @@ public class TimelineWebServices {
    */
   @GET
   @Path("/{entityType}/events")
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelineEvents getEvents(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -214,6 +219,8 @@ public class TimelineWebServices {
    */
   @POST
   @Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelinePutResponse postEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -243,6 +250,8 @@ public class TimelineWebServices {
   @PUT
   @Path("/domain")
   @Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelinePutResponse putDomain(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -274,7 +283,8 @@ public class TimelineWebServices {
    */
   @GET
   @Path("/domain/{domainId}")
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelineDomain getDomain(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
@@ -305,7 +315,8 @@ public class TimelineWebServices {
    */
   @GET
   @Path("/domain")
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8
+      /* , MediaType.APPLICATION_XML */})
   public TimelineDomains getDomains(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 4940c50..c575e70 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
@@ -43,6 +43,7 @@ import javax.ws.rs.core.MediaType;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
@@ -314,7 +315,8 @@ public class TestAHSWebServices extends JerseyTestBase {
         .path("ws").path("v1").path("applicationhistory").path("about")
         .queryParam("user.name", USERS[round])
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineAbout actualAbout = response.getEntity(TimelineAbout.class);
     TimelineAbout expectedAbout =
         TimelineUtils.createTimelineAbout("Generic History Service API");
@@ -343,7 +345,8 @@ public class TestAHSWebServices extends JerseyTestBase {
           .queryParam("state", YarnApplicationState.FINISHED.toString())
           .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject apps = json.getJSONObject("apps");
@@ -362,7 +365,8 @@ public class TestAHSWebServices extends JerseyTestBase {
           .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject app = json.getJSONObject("app");
@@ -391,7 +395,8 @@ public class TestAHSWebServices extends JerseyTestBase {
       assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject appAttempts = json.getJSONObject("appAttempts");
@@ -417,7 +422,8 @@ public class TestAHSWebServices extends JerseyTestBase {
       assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject appAttempt = json.getJSONObject("appAttempt");
@@ -446,7 +452,8 @@ public class TestAHSWebServices extends JerseyTestBase {
       assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject containers = json.getJSONObject("containers");
@@ -474,7 +481,8 @@ public class TestAHSWebServices extends JerseyTestBase {
       assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject container = json.getJSONObject("container");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestRollingLevelDBTimelineStore.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/TestRollingLevelDBTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestRollingLevelDBTimelineStore.java
index 956e9e9..ab5d7ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestRollingLevelDBTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestRollingLevelDBTimelineStore.java
@@ -42,7 +42,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 /** Test class to verify RollingLevelDBTimelineStore. */
 @InterfaceAudience.Private
@@ -368,7 +368,7 @@ public class TestRollingLevelDBTimelineStore extends TimelineStoreTestUtils {
     long start = System.currentTimeMillis();
     int num = 1000000;
 
-    Log.info("Start test for " + num);
+    Log.getLog().info("Start test for " + num);
 
     final String tezTaskAttemptId = "TEZ_TA";
     final String tezEntityId = "attempt_1429158534256_0001_1_00_000000_";
@@ -414,7 +414,7 @@ public class TestRollingLevelDBTimelineStore extends TimelineStoreTestUtils {
     }
 
     long duration = System.currentTimeMillis() - start;
-    Log.info("Duration for " + num + ": " + duration);
+    Log.getLog().info("Duration for " + num + ": " + duration);
   }
 
   public static void main(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 6fd39dc..80d801d 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
@@ -40,6 +40,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
@@ -120,7 +121,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
           PseudoAuthenticationHandler.ANONYMOUS_ALLOWED)).thenReturn("true");
       ServletContext context = mock(ServletContext.class);
       when(filterConfig.getServletContext()).thenReturn(context);
-      Enumeration<Object> names = mock(Enumeration.class);
+      Enumeration<String> names = mock(Enumeration.class);
       when(names.hasMoreElements()).thenReturn(true, true, true, false);
       when(names.nextElement()).thenReturn(
           AuthenticationFilter.AUTH_TYPE,
@@ -187,7 +188,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("timeline")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineAbout actualAbout = response.getEntity(TimelineAbout.class);
     TimelineAbout expectedAbout =
         TimelineUtils.createTimelineAbout("Timeline API");
@@ -244,7 +246,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     verifyEntities(response.getEntity(TimelineEntities.class));
   }
 
@@ -255,7 +258,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").queryParam("fromId", "id_2")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertEquals(2, response.getEntity(TimelineEntities.class).getEntities()
         .size());
 
@@ -263,7 +267,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").queryParam("fromId", "id_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertEquals(3, response.getEntity(TimelineEntities.class).getEntities()
         .size());
   }
@@ -275,7 +280,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").queryParam("fromTs", Long.toString(beforeTime))
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertEquals(0, response.getEntity(TimelineEntities.class).getEntities()
         .size());
 
@@ -284,7 +290,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
             System.currentTimeMillis()))
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertEquals(3, response.getEntity(TimelineEntities.class).getEntities()
         .size());
   }
@@ -296,7 +303,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").queryParam("primaryFilter", "user:username")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     verifyEntities(response.getEntity(TimelineEntities.class));
   }
 
@@ -308,7 +316,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
             "appname:" + Integer.toString(Integer.MAX_VALUE))
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     verifyEntities(response.getEntity(TimelineEntities.class));
   }
 
@@ -320,7 +329,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
             "long:" + Long.toString((long) Integer.MAX_VALUE + 1l))
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     verifyEntities(response.getEntity(TimelineEntities.class));
   }
 
@@ -333,7 +343,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").queryParam("primaryFilter", "other:123abc")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertEquals(0, response.getEntity(TimelineEntities.class).getEntities()
         .size());
   }
@@ -345,7 +356,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").queryParam("primaryFilter", "other:\"123abc\"")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     verifyEntities(response.getEntity(TimelineEntities.class));
   }
 
@@ -358,7 +370,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
             "user:username,appname:" + Integer.toString(Integer.MAX_VALUE))
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     verifyEntities(response.getEntity(TimelineEntities.class));
   }
 
@@ -369,7 +382,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").path("id_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineEntity entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("id_1", entity.getEntityId());
@@ -387,7 +401,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("type_1").path("id_1").queryParam("fields", "events,otherinfo")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineEntity entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("id_1", entity.getEntityId());
@@ -406,7 +421,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
             "primaryfilters,relatedentities")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineEntity entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("id_1", entity.getEntityId());
@@ -425,7 +441,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .queryParam("entityId", "id_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineEvents events = response.getEntity(TimelineEvents.class);
     Assert.assertNotNull(events);
     Assert.assertEquals(1, events.getAllEvents().size());
@@ -479,7 +496,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
 
     response = r.path("ws").path("v1").path("timeline")
@@ -487,7 +505,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelinePutResponse putResposne =
         response.getEntity(TimelinePutResponse.class);
     Assert.assertNotNull(putResposne);
@@ -497,7 +516,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("test type 1").path("test id 1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("test id 1", entity.getEntityId());
@@ -520,7 +540,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("timeline")
         .queryParam("user.name", "tester").accept(MediaType.APPLICATION_JSON)
          .type(MediaType.APPLICATION_JSON).post(ClientResponse.class, entities);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
   }
 
@@ -542,7 +563,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelinePutResponse putResponse =
           response.getEntity(TimelinePutResponse.class);
       Assert.assertNotNull(putResponse);
@@ -554,7 +576,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       putResponse = response.getEntity(TimelinePutResponse.class);
       Assert.assertNotNull(putResponse);
       Assert.assertEquals(1, putResponse.getErrors().size());
@@ -577,7 +600,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       putResponse = response.getEntity(TimelinePutResponse.class);
       Assert.assertNotNull(putResponse);
       Assert.assertEquals(1, putResponse.getErrors().size());
@@ -591,7 +615,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "reader_user_3")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       entity = response.getEntity(TimelineEntity.class);
       Assert.assertNotNull(entity);
       Assert.assertEquals("test id 3", entity.getEntityId());
@@ -618,7 +643,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelinePutResponse putResposne =
           response.getEntity(TimelinePutResponse.class);
       Assert.assertNotNull(putResposne);
@@ -629,7 +655,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "any_body_2")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       entity = response.getEntity(TimelineEntity.class);
       Assert.assertNotNull(entity);
       Assert.assertEquals("test id 7", entity.getEntityId());
@@ -659,7 +686,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelinePutResponse putResponse =
           response.getEntity(TimelinePutResponse.class);
       Assert.assertEquals(0, putResponse.getErrors().size());
@@ -670,7 +698,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "reader_user_1")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       entity = response.getEntity(TimelineEntity.class);
       Assert.assertNull(entity.getPrimaryFilters().get(
           TimelineStore.SystemFilter.ENTITY_OWNER.toString()));
@@ -681,7 +710,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "reader_user_1")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       entity = response.getEntity(TimelineEntity.class);
       Assert.assertNull(entity.getPrimaryFilters().get(
           TimelineStore.SystemFilter.ENTITY_OWNER.toString()));
@@ -692,7 +722,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "reader_user_1")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       entity = response.getEntity(TimelineEntity.class);
       Assert.assertNull(entity.getPrimaryFilters().get(
           TimelineStore.SystemFilter.ENTITY_OWNER.toString()));
@@ -703,7 +734,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "reader_user_2")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
@@ -729,7 +761,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelinePutResponse putResponse =
           response.getEntity(TimelinePutResponse.class);
       Assert.assertEquals(0, putResponse.getErrors().size());
@@ -748,7 +781,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       putResponse = response.getEntity(TimelinePutResponse.class);
       Assert.assertEquals(0, putResponse.getErrors().size());
 
@@ -758,7 +792,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .path("test type 4")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       entities = response.getEntity(TimelineEntities.class);
       // Reader 1 should just have the access to entity [4, 4]
       assertEquals(1, entities.getEntities().size());
@@ -792,7 +827,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelinePutResponse putResponse =
           response.getEntity(TimelinePutResponse.class);
       Assert.assertEquals(0, putResponse.getErrors().size());
@@ -815,7 +851,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       putResponse = response.getEntity(TimelinePutResponse.class);
       Assert.assertEquals(0, putResponse.getErrors().size());
 
@@ -826,7 +863,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("entityId", "test id 5,test id 6")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelineEvents events = response.getEntity(TimelineEvents.class);
       // Reader 1 should just have the access to the events of entity [5, 5]
       assertEquals(1, events.getAllEvents().size());
@@ -843,7 +881,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("domain").path("domain_id_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineDomain domain = response.getEntity(TimelineDomain.class);
     verifyDomain(domain, "domain_id_1");
   }
@@ -859,7 +898,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "owner_1")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelineDomain domain = response.getEntity(TimelineDomain.class);
       verifyDomain(domain, "domain_id_1");
 
@@ -868,7 +908,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
@@ -883,7 +924,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .queryParam("owner", "owner_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     TimelineDomains domains = response.getEntity(TimelineDomains.class);
     Assert.assertEquals(2, domains.getDomains().size());
     for (int i = 0; i < domains.getDomains().size(); ++i) {
@@ -903,7 +945,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "owner_1")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       TimelineDomains domains = response.getEntity(TimelineDomains.class);
       Assert.assertEquals(2, domains.getDomains().size());
       for (int i = 0; i < domains.getDomains().size(); ++i) {
@@ -917,7 +960,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       domains = response.getEntity(TimelineDomains.class);
       Assert.assertEquals(0, domains.getDomains().size());
     } finally {
@@ -936,7 +980,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .put(ClientResponse.class, domain);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
 
     response = r.path("ws").path("v1")
@@ -952,7 +997,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("domain").path("test_domain_id")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     domain = response.getEntity(TimelineDomain.class);
     Assert.assertNotNull(domain);
     Assert.assertEquals("test_domain_id", domain.getId());
@@ -974,7 +1020,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .path("domain").path("test_domain_id")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     domain = response.getEntity(TimelineDomain.class);
     Assert.assertNotNull(domain);
     Assert.assertEquals("test_domain_id", domain.getId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
index f10fa08..6fbe39e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
@@ -66,7 +66,7 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
     <dependency>
       <groupId>commons-codec</groupId>
@@ -81,7 +81,7 @@
       <artifactId>jersey-client</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index afbf0d4..a32f06f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -40,6 +40,7 @@ import javax.ws.rs.core.UriInfo;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -95,14 +96,16 @@ public class NMWebServices {
   }
 
   @GET
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodeInfo get() {
     return getNodeInfo();
   }
 
   @GET
   @Path("/info")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public NodeInfo getNodeInfo() {
     init();
     return new NodeInfo(this.nmContext, this.rview);
@@ -110,7 +113,8 @@ public class NMWebServices {
 
   @GET
   @Path("/apps")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppsInfo getNodeApps(@QueryParam("state") String stateQuery,
       @QueryParam("user") String userQuery) {
     init();
@@ -141,7 +145,8 @@ public class NMWebServices {
 
   @GET
   @Path("/apps/{appid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppInfo getNodeApp(@PathParam("appid") String appId) {
     init();
     ApplicationId id = WebAppUtils.parseApplicationId(recordFactory, appId);
@@ -155,7 +160,8 @@ public class NMWebServices {
 
   @GET
   @Path("/containers")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ContainersInfo getNodeContainers(@javax.ws.rs.core.Context
       HttpServletRequest hsr) {
     init();
@@ -175,7 +181,8 @@ public class NMWebServices {
 
   @GET
   @Path("/containers/{containerid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ContainerInfo getNodeContainer(@javax.ws.rs.core.Context
       HttpServletRequest hsr, @PathParam("containerid") String id) {
     ContainerId containerId = null;
@@ -207,7 +214,8 @@ public class NMWebServices {
    */
   @GET
   @Path("/containers/{containerid}/logs")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ContainerLogsInfo getContainerLogsInfo(@javax.ws.rs.core.Context
       HttpServletRequest hsr,
       @PathParam("containerid") String containerIdStr) {
@@ -246,7 +254,7 @@ public class NMWebServices {
    */
   @GET
   @Path("/containers/{containerid}/logs/{filename}")
-  @Produces({ MediaType.TEXT_PLAIN })
+  @Produces({ MediaType.TEXT_PLAIN + "; " + JettyUtils.UTF_8 })
   @Public
   @Unstable
   public Response getContainerLogFile(
@@ -277,7 +285,7 @@ public class NMWebServices {
    */
   @GET
   @Path("/containerlogs/{containerid}/{filename}")
-  @Produces({ MediaType.TEXT_PLAIN })
+  @Produces({ MediaType.TEXT_PLAIN + "; " + JettyUtils.UTF_8 })
   @Public
   @Unstable
   public Response getLogs(@PathParam("containerid") String containerIdStr,
@@ -363,7 +371,7 @@ public class NMWebServices {
         }
       };
       ResponseBuilder resp = Response.ok(stream);
-      resp.header("Content-Type", contentType);
+      resp.header("Content-Type", contentType + "; " + JettyUtils.UTF_8);
       // Sending the X-Content-Type-Options response header with the value
       // nosniff will prevent Internet Explorer from MIME-sniffing a response
       // away from the declared content-type.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.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/amrmproxy/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
index 2ccf827..f02e306 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
@@ -113,7 +113,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 /**
  * Mock Resource Manager facade implementation that exposes all the methods
@@ -156,7 +156,7 @@ public class MockResourceManagerFacade implements
       RegisterApplicationMasterRequest request) throws YarnException,
       IOException {
     String amrmToken = getAppIdentifier();
-    Log.info("Registering application attempt: " + amrmToken);
+    Log.getLog().info("Registering application attempt: " + amrmToken);
 
     synchronized (applicationContainerIdMap) {
       Assert.assertFalse("The application id is already registered: "
@@ -175,7 +175,7 @@ public class MockResourceManagerFacade implements
       FinishApplicationMasterRequest request) throws YarnException,
       IOException {
     String amrmToken = getAppIdentifier();
-    Log.info("Finishing application attempt: " + amrmToken);
+    Log.getLog().info("Finishing application attempt: " + amrmToken);
 
     synchronized (applicationContainerIdMap) {
       // Remove the containers that were being tracked for this application
@@ -252,7 +252,8 @@ public class MockResourceManagerFacade implements
 
     if (request.getReleaseList() != null
         && request.getReleaseList().size() > 0) {
-      Log.info("Releasing containers: " + request.getReleaseList().size());
+      Log.getLog().info("Releasing containers: "
+          + request.getReleaseList().size());
       synchronized (applicationContainerIdMap) {
         Assert.assertTrue(
             "The application id is not registered before allocate(): "
@@ -292,7 +293,7 @@ public class MockResourceManagerFacade implements
       }
     }
 
-    Log.info("Allocating containers: " + containerList.size()
+    Log.getLog().info("Allocating containers: " + containerList.size()
         + " for application attempt: " + conf.get("AMRMTOKEN"));
     return AllocateResponse.newInstance(0,
         new ArrayList<ContainerStatus>(), containerList,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index b9d18a3..e7330aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -132,7 +132,7 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
-import org.mortbay.util.MultiException;
+import org.eclipse.jetty.util.MultiException;
 
 import com.google.common.base.Supplier;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 80ccf53..49e410e 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
@@ -31,6 +31,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
@@ -233,7 +234,8 @@ public class TestNMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeInfo(json);
   }
@@ -244,7 +246,8 @@ public class TestNMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node/")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeInfo(json);
   }
@@ -256,7 +259,8 @@ public class TestNMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node")
         .get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeInfo(json);
   }
@@ -266,7 +270,8 @@ public class TestNMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("node").path("info")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeInfo(json);
   }
@@ -277,7 +282,8 @@ public class TestNMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node")
         .path("info/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeInfo(json);
   }
@@ -288,7 +294,8 @@ public class TestNMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("node").path("info")
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeInfo(json);
   }
@@ -299,7 +306,8 @@ public class TestNMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node")
         .path("info/").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML+ "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -400,7 +408,7 @@ public class TestNMWebServices extends JerseyTestBase {
         .queryParam("size", "-10000")
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
     responseText = response.getEntity(String.class);
-    assertEquals("text/plain", response.getType().toString());
+    assertEquals("text/plain; charset=utf-8", response.getType().toString());
     assertEquals(fullTextSize, responseText.getBytes().length);
     assertEquals(logMessage, responseText);
 
@@ -411,7 +419,8 @@ public class TestNMWebServices extends JerseyTestBase {
     responseText = response.getEntity(String.class);
     assertEquals(logMessage, responseText);
     assertEquals(200, response.getStatus());
-    assertEquals("application/octet-stream", response.getType().toString());
+    assertEquals("application/octet-stream; charset=utf-8",
+        response.getType().toString());
 
     // specify a invalid format value
     response = r.path(filename)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 3b7694a..6316282 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
@@ -34,6 +34,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -182,7 +183,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("apps isn't empty",
         new JSONObject().toString(), json.get("apps").toString());
@@ -241,7 +243,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
 
     ClientResponse response = r.path("ws").path("v1").path("node").path(path)
         .accept(media).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     JSONObject info = json.getJSONObject("apps");
     assertEquals("incorrect number of elements", 1, info.length());
@@ -270,7 +273,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .queryParam("user", "mockUser").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     JSONObject info = json.getJSONObject("apps");
@@ -293,7 +297,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .queryParam("user", "george").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("apps is not empty",
         new JSONObject().toString(), json.get("apps").toString());
@@ -317,7 +322,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -351,7 +357,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
         .queryParam("state", ApplicationState.RUNNING.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     JSONObject info = json.getJSONObject("apps");
@@ -375,7 +382,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .queryParam("state", ApplicationState.INITING.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
 
     assertEquals("apps is not empty",
@@ -401,7 +409,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -431,7 +440,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -462,7 +472,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       ClientResponse response = ue.getResponse();
 
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String msg = response.getEntity(String.class);
 
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -517,7 +528,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .path(app.getAppId().toString()).accept(media)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeAppInfo(json.getJSONObject("app"), app, hash);
   }
@@ -535,7 +547,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
         .path(app.getAppId().toString() + "/")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     verifyNodeAppInfo(json.getJSONObject("app"), app, hash);
   }
@@ -557,7 +570,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -594,7 +608,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -623,7 +638,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
 
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();
@@ -647,7 +663,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node").path("apps")
         .path(app.getAppId().toString() + "/")
         .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 06a3995..36acfe0 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
@@ -39,6 +39,7 @@ 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.http.JettyUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -194,7 +195,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node")
         .path("containers").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("apps isn't empty",
         new JSONObject().toString(), json.get("containers").toString());
@@ -269,7 +271,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
 
     ClientResponse response = r.path("ws").path("v1").path("node").path(path)
         .accept(media).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     JSONObject info = json.getJSONObject("containers");
     assertEquals("incorrect number of elements", 1, info.length());
@@ -313,7 +316,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     for (String id : hash.keySet()) {
       ClientResponse response = r.path("ws").path("v1").path("node")
           .path("containers").path(id).accept(media).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       verifyNodeContainerInfo(json.getJSONObject("container"), nmContext
           .getContainers().get(ContainerId.fromString(id)));
@@ -337,7 +341,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -371,7 +376,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -405,7 +411,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
       assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
       assertEquals("incorrect number of elements", 3, exception.length());
@@ -439,7 +446,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("node")
           .path("containers").path(id).accept(MediaType.APPLICATION_XML)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -467,7 +475,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("node")
         .path("containers").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 56fb075..8e83a15 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -35,7 +35,7 @@
   <dependencies>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
@@ -143,10 +143,14 @@
       <artifactId>jersey-client</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>


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


[14/50] [abbrv] hadoop git commit: YARN-4388. Cleanup mapreduce.job.hdfs-servers from yarn-default.xml (Junping Du via Varun Saxena)

Posted by vi...@apache.org.
YARN-4388. Cleanup mapreduce.job.hdfs-servers from yarn-default.xml (Junping Du via Varun Saxena)


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

Branch: refs/heads/HDFS-9806
Commit: dd4ed6a587bf9cc57eb38d7957d8a907901a1cac
Parents: 7e3c327
Author: Varun Saxena <va...@apache.org>
Authored: Fri Oct 28 01:41:14 2016 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Oct 28 02:22:25 2016 +0530

----------------------------------------------------------------------
 .../src/main/resources/mapred-default.xml                       | 5 +++++
 .../apache/hadoop/yarn/conf/TestYarnConfigurationFields.java    | 4 ----
 .../hadoop-yarn-common/src/main/resources/yarn-default.xml      | 5 -----
 3 files changed, 5 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4ed6a5/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 fe29212..2b834bd 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
@@ -24,6 +24,11 @@
 <configuration>
 
 <property>
+  <name>mapreduce.job.hdfs-servers</name>
+  <value>${fs.defaultFS}</value>
+</property>
+
+<property>
   <name>mapreduce.job.committer.setup.cleanup.needed</name>
   <value>true</value>
   <description> true, if job needs job-setup and job-cleanup.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4ed6a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 668821d..0c40fa9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -135,10 +135,6 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     xmlPropsToSkipCompare = new HashSet<String>();
     xmlPrefixToSkipCompare = new HashSet<String>();
 
-    // Should probably be moved from yarn-default.xml to mapred-default.xml
-    xmlPropsToSkipCompare.add("mapreduce.job.hdfs-servers");
-    xmlPropsToSkipCompare.add("mapreduce.job.jar");
-
     // Possibly obsolete, but unable to verify 100%
     xmlPropsToSkipCompare.add("yarn.nodemanager.aux-services.mapreduce_shuffle.class");
     xmlPropsToSkipCompare.add("yarn.resourcemanager.container.liveness-monitor.interval-ms");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4ed6a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index c7076e5..6c247b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1801,11 +1801,6 @@
   <!-- Map Reduce Configuration -->
 
   <property>
-    <name>mapreduce.job.hdfs-servers</name>
-    <value>${fs.defaultFS}</value>
-  </property>
-
-  <property>
     <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
     <value>org.apache.hadoop.mapred.ShuffleHandler</value>
   </property>


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


[08/50] [abbrv] hadoop git commit: Revert "HDFS-10455. Logging the username when deny the setOwner operation. Contributed by Tianyin Xiu"

Posted by vi...@apache.org.
Revert "HDFS-10455. Logging the username when deny the setOwner operation. Contributed by Tianyin Xiu"

This reverts commit ac35ee9393e0afce9fede1d2052e7bf4032312fd.


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

Branch: refs/heads/HDFS-9806
Commit: db41965996630c8f8c48618c5a0bc24321320c52
Parents: 79aeddc
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Oct 27 14:07:28 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Oct 27 14:07:28 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java    | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db419659/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 488c600..417ce01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -84,12 +84,10 @@ public class FSDirAttrOp {
       fsd.checkOwner(pc, iip);
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
-          throw new AccessControlException("User " + pc.getUser()
-              + " is not a super user (non-super user cannot change owner).");
+          throw new AccessControlException("Non-super user cannot change owner");
         }
         if (group != null && !pc.isMemberOfGroup(group)) {
-          throw new AccessControlException(
-              "User " + pc.getUser() + " does not belong to " + group);
+          throw new AccessControlException("User does not belong to " + group);
         }
       }
       unprotectedSetOwner(fsd, iip, username, group);


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


[12/50] [abbrv] hadoop git commit: HDFS-11055. Update default-log4j.properties for httpfs to imporve test logging. Contributed by Wei-Chiu Chuang.

Posted by vi...@apache.org.
HDFS-11055. Update default-log4j.properties for httpfs to imporve test logging. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/HDFS-9806
Commit: 31ff42b51037632ec871f29efc0fa894e1b738d0
Parents: b4a8fbc
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Thu Oct 27 13:37:00 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Thu Oct 27 13:37:00 2016 -0700

----------------------------------------------------------------------
 .../src/test/resources/default-log4j.properties              | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ff42b5/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/default-log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/default-log4j.properties b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/default-log4j.properties
index 7517512..45a8412 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/default-log4j.properties
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/default-log4j.properties
@@ -18,5 +18,9 @@ log4j.appender.test.File=${test.dir}/test.log
 log4j.appender.test.Append=true
 log4j.appender.test.layout=org.apache.log4j.PatternLayout
 log4j.appender.test.layout.ConversionPattern=%d{ISO8601} %5p %20c{1}: %4L - %m%n
-log4j.rootLogger=ALL, test
-
+log4j.rootLogger=INFO, stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{1} - %m%n
+log4j.logger.com.sun.jersey.server.wadl.generators.AbstractWadlGeneratorGrammarGenerator=OFF


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


[15/50] [abbrv] hadoop git commit: YARN-5776. Checkstyle: MonitoringThread.Run method length is too long (miklos.szegedi@cloudera.com via rkanter)

Posted by vi...@apache.org.
YARN-5776. Checkstyle: MonitoringThread.Run method length is too long (miklos.szegedi@cloudera.com via rkanter)


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

Branch: refs/heads/HDFS-9806
Commit: 9449519a2503c55d9eac8fd7519df28aa0760059
Parents: dd4ed6a
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Oct 27 14:36:27 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Oct 27 14:36:38 2016 -0700

----------------------------------------------------------------------
 .../monitor/ContainersMonitorImpl.java          | 460 +++++++++++--------
 1 file changed, 279 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9449519a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index a04a914..cd9d6af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -48,10 +48,14 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
+/**
+ * Monitors containers collecting resource usage and preempting the container
+ * if it exceeds its limits.
+ */
 public class ContainersMonitorImpl extends AbstractService implements
     ContainersMonitor {
 
-  final static Log LOG = LogFactory
+  private final static Log LOG = LogFactory
       .getLog(ContainersMonitorImpl.class);
 
   private long monitoringInterval;
@@ -66,7 +70,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   private final ContainerExecutor containerExecutor;
   private final Dispatcher eventDispatcher;
-  protected final Context context;
+  private final Context context;
   private ResourceCalculatorPlugin resourceCalculatorPlugin;
   private Configuration conf;
   private static float vmemRatio;
@@ -84,15 +88,18 @@ public class ContainersMonitorImpl extends AbstractService implements
   private static final long UNKNOWN_MEMORY_LIMIT = -1L;
   private int nodeCpuPercentageForYARN;
 
+  /**
+   * Type of container metric.
+   */
   @Private
-  public static enum ContainerMetric {
+  public enum ContainerMetric {
     CPU, MEMORY
   }
 
   private ResourceUtilization containersUtilization;
   // Tracks the aggregated allocation of the currently allocated containers
   // when queuing of containers at the NMs is enabled.
-  private ResourceUtilization containersAllocation;
+  private final ResourceUtilization containersAllocation;
 
   private volatile boolean stopped = false;
 
@@ -111,44 +118,47 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   @Override
-  protected void serviceInit(Configuration conf) throws Exception {
+  protected void serviceInit(Configuration myConf) throws Exception {
+    this.conf = myConf;
     this.monitoringInterval =
-        conf.getLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS,
-            conf.getLong(YarnConfiguration.NM_RESOURCE_MON_INTERVAL_MS,
+        this.conf.getLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS,
+            this.conf.getLong(YarnConfiguration.NM_RESOURCE_MON_INTERVAL_MS,
                 YarnConfiguration.DEFAULT_NM_RESOURCE_MON_INTERVAL_MS));
 
     Class<? extends ResourceCalculatorPlugin> clazz =
-        conf.getClass(YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
-            conf.getClass(
+        this.conf.getClass(YarnConfiguration
+                        .NM_CONTAINER_MON_RESOURCE_CALCULATOR,
+            this.conf.getClass(
                 YarnConfiguration.NM_MON_RESOURCE_CALCULATOR, null,
                 ResourceCalculatorPlugin.class),
             ResourceCalculatorPlugin.class);
     this.resourceCalculatorPlugin =
-        ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, conf);
+        ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, this.conf);
     LOG.info(" Using ResourceCalculatorPlugin : "
         + this.resourceCalculatorPlugin);
-    processTreeClass = conf.getClass(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, null,
+    processTreeClass = this.conf.getClass(
+            YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, null,
             ResourceCalculatorProcessTree.class);
-    this.conf = conf;
     LOG.info(" Using ResourceCalculatorProcessTree : "
         + this.processTreeClass);
 
     this.containerMetricsEnabled =
-        conf.getBoolean(YarnConfiguration.NM_CONTAINER_METRICS_ENABLE,
+        this.conf.getBoolean(YarnConfiguration.NM_CONTAINER_METRICS_ENABLE,
             YarnConfiguration.DEFAULT_NM_CONTAINER_METRICS_ENABLE);
     this.containerMetricsPeriodMs =
-        conf.getLong(YarnConfiguration.NM_CONTAINER_METRICS_PERIOD_MS,
+        this.conf.getLong(YarnConfiguration.NM_CONTAINER_METRICS_PERIOD_MS,
             YarnConfiguration.DEFAULT_NM_CONTAINER_METRICS_PERIOD_MS);
-    this.containerMetricsUnregisterDelayMs = conf.getLong(
+    this.containerMetricsUnregisterDelayMs = this.conf.getLong(
         YarnConfiguration.NM_CONTAINER_METRICS_UNREGISTER_DELAY_MS,
         YarnConfiguration.DEFAULT_NM_CONTAINER_METRICS_UNREGISTER_DELAY_MS);
 
     long configuredPMemForContainers =
         NodeManagerHardwareUtils.getContainerMemoryMB(
-            this.resourceCalculatorPlugin, conf) * 1024 * 1024L;
+            this.resourceCalculatorPlugin, this.conf) * 1024 * 1024L;
 
     long configuredVCoresForContainers =
-        NodeManagerHardwareUtils.getVCores(this.resourceCalculatorPlugin, conf);
+        NodeManagerHardwareUtils.getVCores(this.resourceCalculatorPlugin,
+            this.conf);
 
     // Setting these irrespective of whether checks are enabled. Required in
     // the UI.
@@ -157,16 +167,18 @@ public class ContainersMonitorImpl extends AbstractService implements
     this.maxVCoresAllottedForContainers = configuredVCoresForContainers;
 
     // ///////// Virtual memory configuration //////
-    vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO,
+    vmemRatio = this.conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO,
         YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
     Preconditions.checkArgument(vmemRatio > 0.99f,
         YarnConfiguration.NM_VMEM_PMEM_RATIO + " should be at least 1.0");
     this.maxVmemAllottedForContainers =
         (long) (vmemRatio * configuredPMemForContainers);
 
-    pmemCheckEnabled = conf.getBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED,
+    pmemCheckEnabled = this.conf.getBoolean(
+        YarnConfiguration.NM_PMEM_CHECK_ENABLED,
         YarnConfiguration.DEFAULT_NM_PMEM_CHECK_ENABLED);
-    vmemCheckEnabled = conf.getBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED,
+    vmemCheckEnabled = this.conf.getBoolean(
+        YarnConfiguration.NM_VMEM_CHECK_ENABLED,
         YarnConfiguration.DEFAULT_NM_VMEM_CHECK_ENABLED);
     LOG.info("Physical memory check enabled: " + pmemCheckEnabled);
     LOG.info("Virtual memory check enabled: " + vmemCheckEnabled);
@@ -175,7 +187,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     LOG.info("ContainersMonitor enabled: " + containersMonitorEnabled);
 
     nodeCpuPercentageForYARN =
-        NodeManagerHardwareUtils.getNodeCpuPercentage(conf);
+        NodeManagerHardwareUtils.getNodeCpuPercentage(this.conf);
 
     if (pmemCheckEnabled) {
       // Logging if actual pmem cannot be determined.
@@ -201,7 +213,7 @@ public class ContainersMonitorImpl extends AbstractService implements
                 1) + "). Thrashing might happen.");
       }
     }
-    super.serviceInit(conf);
+    super.serviceInit(this.conf);
   }
 
   private boolean isContainerMonitorEnabled() {
@@ -241,12 +253,15 @@ public class ContainersMonitorImpl extends AbstractService implements
       try {
         this.monitoringThread.join();
       } catch (InterruptedException e) {
-        ;
+        LOG.info("ContainersMonitorImpl monitoring thread interrupted");
       }
     }
     super.serviceStop();
   }
 
+  /**
+   * Encapsulates resource requirements of a process and its tree.
+   */
   public static class ProcessTreeInfo {
     private ContainerId containerId;
     private String pid;
@@ -278,49 +293,49 @@ public class ContainersMonitorImpl extends AbstractService implements
       this.pid = pid;
     }
 
-    public ResourceCalculatorProcessTree getProcessTree() {
+    ResourceCalculatorProcessTree getProcessTree() {
       return this.pTree;
     }
 
-    public void setProcessTree(ResourceCalculatorProcessTree pTree) {
-      this.pTree = pTree;
+    void setProcessTree(ResourceCalculatorProcessTree mypTree) {
+      this.pTree = mypTree;
     }
 
     /**
      * @return Virtual memory limit for the process tree in bytes
      */
-    public synchronized long getVmemLimit() {
+    synchronized long getVmemLimit() {
       return this.vmemLimit;
     }
 
     /**
      * @return Physical memory limit for the process tree in bytes
      */
-    public synchronized long getPmemLimit() {
+    synchronized long getPmemLimit() {
       return this.pmemLimit;
     }
 
     /**
      * @return Number of cpu vcores assigned
      */
-    public synchronized int getCpuVcores() {
+    synchronized int getCpuVcores() {
       return this.cpuVcores;
     }
 
     /**
-     * Set resource limit for enforcement
-     * @param pmemLimit
+     * Set resource limit for enforcement.
+     * @param myPmemLimit
      *          Physical memory limit for the process tree in bytes
-     * @param vmemLimit
+     * @param myVmemLimit
      *          Virtual memory limit for the process tree in bytes
-     * @param cpuVcores
+     * @param myCpuVcores
      *          Number of cpu vcores assigned
      */
-    public synchronized void setResourceLimit(
-        long pmemLimit, long vmemLimit, int cpuVcores) {
-      this.pmemLimit = pmemLimit;
-      this.vmemLimit = vmemLimit;
-      this.cpuVcores = cpuVcores;
+    synchronized void setResourceLimit(
+        long myPmemLimit, long myVmemLimit, int myCpuVcores) {
+      this.pmemLimit = myPmemLimit;
+      this.vmemLimit = myVmemLimit;
+      this.cpuVcores = myCpuVcores;
     }
   }
 
@@ -354,7 +369,7 @@ public class ContainersMonitorImpl extends AbstractService implements
    *         or if processes in the tree, older than this thread's monitoring
    *         interval, exceed the memory limit. False, otherwise.
    */
-  boolean isProcessTreeOverLimit(String containerId,
+  private boolean isProcessTreeOverLimit(String containerId,
                                   long currentMemUsage,
                                   long curMemUsageOfAgedProcesses,
                                   long vmemLimit) {
@@ -388,7 +403,7 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   private class MonitoringThread extends Thread {
-    public MonitoringThread() {
+    MonitoringThread() {
       super("Container Monitor");
     }
 
@@ -425,43 +440,8 @@ public class ContainersMonitorImpl extends AbstractService implements
           try {
             String pId = ptInfo.getPID();
 
-            // Initialize any uninitialized processTrees
-            if (pId == null) {
-              // get pid from ContainerId
-              pId = containerExecutor.getProcessId(ptInfo.getContainerId());
-              if (pId != null) {
-                // pId will be null, either if the container is not spawned yet
-                // or if the container's pid is removed from ContainerExecutor
-                LOG.debug("Tracking ProcessTree " + pId
-                    + " for the first time");
-
-                ResourceCalculatorProcessTree pt =
-                    ResourceCalculatorProcessTree.
-                        getResourceCalculatorProcessTree(
-                            pId, processTreeClass, conf);
-                ptInfo.setPid(pId);
-                ptInfo.setProcessTree(pt);
-
-                if (containerMetricsEnabled) {
-                  ContainerMetrics usageMetrics = ContainerMetrics
-                      .forContainer(containerId, containerMetricsPeriodMs,
-                      containerMetricsUnregisterDelayMs);
-                  usageMetrics.recordProcessId(pId);
-                }
-                Container container = context.getContainers().get(containerId);
-                String[] ipAndHost = containerExecutor.getIpAndHost(container);
-                if (ipAndHost != null && ipAndHost[0] != null
-                    && ipAndHost[1] != null) {
-                  container.setIpAndHost(ipAndHost);
-                  LOG.info(containerId + "'s ip = " + ipAndHost[0]
-                      + ", and hostname = " + ipAndHost[1]);
-                } else {
-                  LOG.info("Can not get both ip and hostname: " + Arrays
-                      .toString(ipAndHost));
-                }
-              }
-            }
-            // End of initializing any uninitialized processTrees
+            // Initialize uninitialized process trees
+            initializeProcessTrees(entry);
 
             if (pId == null || !isResourceCalculatorAvailable()) {
               continue; // processTree cannot be tracked
@@ -487,74 +467,11 @@ public class ContainersMonitorImpl extends AbstractService implements
               continue;
             }
 
-            float cpuUsageTotalCoresPercentage = cpuUsagePercentPerCore /
-                resourceCalculatorPlugin.getNumProcessors();
-
-            // Multiply by 1000 to avoid losing data when converting to int
-            int milliVcoresUsed = (int) (cpuUsageTotalCoresPercentage * 1000
-                * maxVCoresAllottedForContainers /nodeCpuPercentageForYARN);
-            // as processes begin with an age 1, we want to see if there
-            // are processes more than 1 iteration old.
-            long curMemUsageOfAgedProcesses = pTree.getVirtualMemorySize(1);
-            long curRssMemUsageOfAgedProcesses = pTree.getRssMemorySize(1);
-            long vmemLimit = ptInfo.getVmemLimit();
-            long pmemLimit = ptInfo.getPmemLimit();
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(String.format(
-                  "Memory usage of ProcessTree %s for container-id %s: ",
-                  pId, containerId.toString()) +
-                  formatUsageString(
-                      currentVmemUsage, vmemLimit,
-                      currentPmemUsage, pmemLimit));
-            }
-
-            // Add resource utilization for this container
-            trackedContainersUtilization.addTo(
-                (int) (currentPmemUsage >> 20),
-                (int) (currentVmemUsage >> 20),
-                milliVcoresUsed / 1000.0f);
-
-            // Add usage to container metrics
-            if (containerMetricsEnabled) {
-              ContainerMetrics.forContainer(
-                  containerId, containerMetricsPeriodMs,
-                  containerMetricsUnregisterDelayMs).recordMemoryUsage(
-                  (int) (currentPmemUsage >> 20));
-              ContainerMetrics.forContainer(
-                  containerId, containerMetricsPeriodMs,
-                  containerMetricsUnregisterDelayMs).recordCpuUsage
-                  ((int)cpuUsagePercentPerCore, milliVcoresUsed);
-            }
+            recordUsage(containerId, pId, pTree, ptInfo, currentVmemUsage,
+                    currentPmemUsage, trackedContainersUtilization);
 
-            boolean isMemoryOverLimit = false;
-            String msg = "";
-            int containerExitStatus = ContainerExitStatus.INVALID;
-            if (isVmemCheckEnabled()
-                && isProcessTreeOverLimit(containerId.toString(),
-                    currentVmemUsage, curMemUsageOfAgedProcesses, vmemLimit)) {
-              // Container (the root process) is still alive and overflowing
-              // memory.
-              // Dump the process-tree and then clean it up.
-              msg = formatErrorMessage("virtual",
-                  currentVmemUsage, vmemLimit,
-                  currentPmemUsage, pmemLimit,
-                  pId, containerId, pTree);
-              isMemoryOverLimit = true;
-              containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_VMEM;
-            } else if (isPmemCheckEnabled()
-                && isProcessTreeOverLimit(containerId.toString(),
-                    currentPmemUsage, curRssMemUsageOfAgedProcesses,
-                    pmemLimit)) {
-              // Container (the root process) is still alive and overflowing
-              // memory.
-              // Dump the process-tree and then clean it up.
-              msg = formatErrorMessage("physical",
-                  currentVmemUsage, vmemLimit,
-                  currentPmemUsage, pmemLimit,
-                  pId, containerId, pTree);
-              isMemoryOverLimit = true;
-              containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_PMEM;
-            }
+            checkLimit(containerId, pId, pTree, ptInfo,
+                    currentVmemUsage, currentPmemUsage);
 
             // Accounting the total memory in usage for all containers
             vmemUsageByAllContainers += currentVmemUsage;
@@ -563,32 +480,8 @@ public class ContainersMonitorImpl extends AbstractService implements
             cpuUsagePercentPerCoreByAllContainers += cpuUsagePercentPerCore;
             cpuUsageTotalCoresByAllContainers += cpuUsagePercentPerCore;
 
-            if (isMemoryOverLimit) {
-              // Virtual or physical memory over limit. Fail the container and
-              // remove
-              // the corresponding process tree
-              LOG.warn(msg);
-              // warn if not a leader
-              if (!pTree.checkPidPgrpidForMatch()) {
-                LOG.error("Killed container process with PID " + pId
-                    + " but it is not a process group leader.");
-              }
-              // kill the container
-              eventDispatcher.getEventHandler().handle(
-                  new ContainerKillEvent(containerId,
-                      containerExitStatus, msg));
-              trackingContainers.remove(containerId);
-              LOG.info("Removed ProcessTree with root " + pId);
-            }
-
-            ContainerImpl container =
-                (ContainerImpl) context.getContainers().get(containerId);
-            NMTimelinePublisher nmMetricsPublisher =
-                container.getNMTimelinePublisher();
-            if (nmMetricsPublisher != null) {
-              nmMetricsPublisher.reportContainerResourceUsage(container,
-                  currentPmemUsage, cpuUsagePercentPerCore);
-            }
+            reportResourceUsage(containerId, currentPmemUsage,
+                    cpuUsagePercentPerCore);
           } catch (Exception e) {
             // Log the exception and proceed to the next container.
             LOG.warn("Uncaught exception in ContainersMonitorImpl "
@@ -617,21 +510,226 @@ public class ContainersMonitorImpl extends AbstractService implements
       }
     }
 
+    /**
+     * Initialize any uninitialized processTrees.
+     * @param entry process tree entry to fill in
+     */
+    private void initializeProcessTrees(
+            Entry<ContainerId, ProcessTreeInfo> entry) {
+      ContainerId containerId = entry.getKey();
+      ProcessTreeInfo ptInfo = entry.getValue();
+      String pId = ptInfo.getPID();
+
+      // Initialize any uninitialized processTrees
+      if (pId == null) {
+        // get pid from ContainerId
+        pId = containerExecutor.getProcessId(ptInfo.getContainerId());
+        if (pId != null) {
+          // pId will be null, either if the container is not spawned yet
+          // or if the container's pid is removed from ContainerExecutor
+          LOG.debug("Tracking ProcessTree " + pId
+                  + " for the first time");
+
+          ResourceCalculatorProcessTree pt =
+                  ResourceCalculatorProcessTree.
+                        getResourceCalculatorProcessTree(
+                            pId, processTreeClass, conf);
+          ptInfo.setPid(pId);
+          ptInfo.setProcessTree(pt);
+
+          if (containerMetricsEnabled) {
+            ContainerMetrics usageMetrics = ContainerMetrics
+                    .forContainer(containerId, containerMetricsPeriodMs,
+                      containerMetricsUnregisterDelayMs);
+            usageMetrics.recordProcessId(pId);
+          }
+
+          Container container = context.getContainers().get(containerId);
+          String[] ipAndHost = containerExecutor.getIpAndHost(container);
+          if (ipAndHost != null && ipAndHost[0] != null
+                  && ipAndHost[1] != null) {
+            container.setIpAndHost(ipAndHost);
+            LOG.info(containerId + "'s ip = " + ipAndHost[0]
+                    + ", and hostname = " + ipAndHost[1]);
+          } else {
+            LOG.info("Can not get both ip and hostname: " + Arrays
+                    .toString(ipAndHost));
+          }
+        }
+      }
+      // End of initializing any uninitialized processTrees
+    }
+
+    /**
+     * Record usage metrics.
+     * @param containerId container id
+     * @param pId process id
+     * @param pTree valid process tree entry with CPU measurement
+     * @param ptInfo process tree info with limit information
+     * @param currentVmemUsage virtual memory measurement
+     * @param currentPmemUsage physical memory measurement
+     * @param trackedContainersUtilization utilization tracker to update
+     */
+    private void recordUsage(ContainerId containerId, String pId,
+                             ResourceCalculatorProcessTree pTree,
+                             ProcessTreeInfo ptInfo,
+                             long currentVmemUsage, long currentPmemUsage,
+                             ResourceUtilization trackedContainersUtilization) {
+      float cpuUsagePercentPerCore = pTree.getCpuUsagePercent();
+      float cpuUsageTotalCoresPercentage = cpuUsagePercentPerCore /
+              resourceCalculatorPlugin.getNumProcessors();
+
+      // Multiply by 1000 to avoid losing data when converting to int
+      int milliVcoresUsed = (int) (cpuUsageTotalCoresPercentage * 1000
+              * maxVCoresAllottedForContainers /nodeCpuPercentageForYARN);
+      long vmemLimit = ptInfo.getVmemLimit();
+      long pmemLimit = ptInfo.getPmemLimit();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format(
+                "Memory usage of ProcessTree %s for container-id %s: ",
+                pId, containerId.toString()) +
+                formatUsageString(
+                      currentVmemUsage, vmemLimit,
+                      currentPmemUsage, pmemLimit));
+      }
+
+      // Add resource utilization for this container
+      trackedContainersUtilization.addTo(
+              (int) (currentPmemUsage >> 20),
+              (int) (currentVmemUsage >> 20),
+              milliVcoresUsed / 1000.0f);
+
+      // Add usage to container metrics
+      if (containerMetricsEnabled) {
+        ContainerMetrics.forContainer(
+                containerId, containerMetricsPeriodMs,
+                containerMetricsUnregisterDelayMs).recordMemoryUsage(
+                (int) (currentPmemUsage >> 20));
+        ContainerMetrics.forContainer(
+                containerId, containerMetricsPeriodMs,
+                containerMetricsUnregisterDelayMs).recordCpuUsage((int)
+                cpuUsagePercentPerCore, milliVcoresUsed);
+      }
+    }
+
+    /**
+     * Check resource limits and take actions if the limit is exceeded.
+     * @param containerId container id
+     * @param pId process id
+     * @param pTree valid process tree entry with CPU measurement
+     * @param ptInfo process tree info with limit information
+     * @param currentVmemUsage virtual memory measurement
+     * @param currentPmemUsage physical memory measurement
+     */
+    @SuppressWarnings("unchecked")
+    private void checkLimit(ContainerId containerId, String pId,
+                            ResourceCalculatorProcessTree pTree,
+                            ProcessTreeInfo ptInfo,
+                            long currentVmemUsage,
+                            long currentPmemUsage) {
+      boolean isMemoryOverLimit = false;
+      long vmemLimit = ptInfo.getVmemLimit();
+      long pmemLimit = ptInfo.getPmemLimit();
+      // as processes begin with an age 1, we want to see if there
+      // are processes more than 1 iteration old.
+      long curMemUsageOfAgedProcesses = pTree.getVirtualMemorySize(1);
+      long curRssMemUsageOfAgedProcesses = pTree.getRssMemorySize(1);
+      String msg = "";
+      int containerExitStatus = ContainerExitStatus.INVALID;
+      if (isVmemCheckEnabled()
+              && isProcessTreeOverLimit(containerId.toString(),
+              currentVmemUsage, curMemUsageOfAgedProcesses, vmemLimit)) {
+        // Container (the root process) is still alive and overflowing
+        // memory.
+        // Dump the process-tree and then clean it up.
+        msg = formatErrorMessage("virtual",
+                formatUsageString(currentVmemUsage, vmemLimit,
+                  currentPmemUsage, pmemLimit),
+                pId, containerId, pTree);
+        isMemoryOverLimit = true;
+        containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_VMEM;
+      } else if (isPmemCheckEnabled()
+              && isProcessTreeOverLimit(containerId.toString(),
+              currentPmemUsage, curRssMemUsageOfAgedProcesses,
+              pmemLimit)) {
+        // Container (the root process) is still alive and overflowing
+        // memory.
+        // Dump the process-tree and then clean it up.
+        msg = formatErrorMessage("physical",
+                formatUsageString(currentVmemUsage, vmemLimit,
+                  currentPmemUsage, pmemLimit),
+                pId, containerId, pTree);
+        isMemoryOverLimit = true;
+        containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_PMEM;
+      }
+
+      if (isMemoryOverLimit) {
+        // Virtual or physical memory over limit. Fail the container and
+        // remove
+        // the corresponding process tree
+        LOG.warn(msg);
+        // warn if not a leader
+        if (!pTree.checkPidPgrpidForMatch()) {
+          LOG.error("Killed container process with PID " + pId
+                  + " but it is not a process group leader.");
+        }
+        // kill the container
+        eventDispatcher.getEventHandler().handle(
+                new ContainerKillEvent(containerId,
+                      containerExitStatus, msg));
+        trackingContainers.remove(containerId);
+        LOG.info("Removed ProcessTree with root " + pId);
+      }
+    }
+
+    /**
+     * Report usage metrics to the timeline service.
+     * @param containerId container id
+     * @param currentPmemUsage physical memory measurement
+     * @param cpuUsagePercentPerCore CPU usage
+     */
+    private void reportResourceUsage(ContainerId containerId,
+        long currentPmemUsage, float cpuUsagePercentPerCore) {
+      ContainerImpl container =
+              (ContainerImpl) context.getContainers().get(containerId);
+      NMTimelinePublisher nmMetricsPublisher =
+              container.getNMTimelinePublisher();
+      if (nmMetricsPublisher != null) {
+        nmMetricsPublisher.reportContainerResourceUsage(container,
+                currentPmemUsage, cpuUsagePercentPerCore);
+      }
+    }
+
+    /**
+     * Format string when memory limit has been exceeded.
+     * @param memTypeExceeded type of memory
+     * @param usageString general memory usage information string
+     * @param pId process id
+     * @param containerId container id
+     * @param pTree process tree to dump full resource utilization graph
+     * @return formatted resource usage information
+     */
     private String formatErrorMessage(String memTypeExceeded,
-        long currentVmemUsage, long vmemLimit,
-        long currentPmemUsage, long pmemLimit,
-        String pId, ContainerId containerId, ResourceCalculatorProcessTree pTree) {
+        String usageString, String pId, ContainerId containerId,
+        ResourceCalculatorProcessTree pTree) {
       return
-        String.format("Container [pid=%s,containerID=%s] is running beyond %s memory limits. ",
+        String.format("Container [pid=%s,containerID=%s] is " +
+            "running beyond %s memory limits. ",
             pId, containerId, memTypeExceeded) +
-        "Current usage: " +
-        formatUsageString(currentVmemUsage, vmemLimit,
-                          currentPmemUsage, pmemLimit) +
+        "Current usage: " + usageString +
         ". Killing container.\n" +
         "Dump of the process-tree for " + containerId + " :\n" +
         pTree.getProcessTreeDump();
     }
 
+    /**
+     * Format memory usage string for reporting.
+     * @param currentVmemUsage virtual memory usage
+     * @param vmemLimit virtual memory limit
+     * @param currentPmemUsage physical memory usage
+     * @param pmemLimit physical memory limit
+     * @return formatted memory information
+     */
     private String formatUsageString(long currentVmemUsage, long vmemLimit,
         long currentPmemUsage, long pmemLimit) {
       return String.format("%sB of %sB physical memory used; " +
@@ -746,7 +844,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     return this.containersUtilization;
   }
 
-  public void setContainersUtilization(ResourceUtilization utilization) {
+  private void setContainersUtilization(ResourceUtilization utilization) {
     this.containersUtilization = utilization;
   }
 
@@ -858,7 +956,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
   }
 
-  protected void onChangeMonitoringContainerResource(
+  private void onChangeMonitoringContainerResource(
       ContainersMonitorEvent monitoringEvent, ContainerId containerId) {
     ChangeMonitoringContainerResourceEvent changeEvent =
         (ChangeMonitoringContainerResourceEvent) monitoringEvent;
@@ -878,14 +976,14 @@ public class ContainersMonitorImpl extends AbstractService implements
     changeContainerResource(containerId, changeEvent.getResource());
   }
 
-  protected void onStopMonitoringContainer(
+  private void onStopMonitoringContainer(
       ContainersMonitorEvent monitoringEvent, ContainerId containerId) {
     LOG.info("Stopping resource-monitoring for " + containerId);
     updateContainerMetrics(monitoringEvent);
     trackingContainers.remove(containerId);
   }
 
-  protected void onStartMonitoringContainer(
+  private void onStartMonitoringContainer(
       ContainersMonitorEvent monitoringEvent, ContainerId containerId) {
     ContainerStartMonitoringEvent startEvent =
         (ContainerStartMonitoringEvent) monitoringEvent;


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


[24/50] [abbrv] hadoop git commit: HDFS-9500. Fix software version counts for DataNodes during rolling upgrade. Contributed by Erik Krogen.

Posted by vi...@apache.org.
HDFS-9500. Fix software version counts for DataNodes during rolling upgrade. Contributed by Erik Krogen.

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

Branch: refs/heads/HDFS-9806
Commit: f3ac1f41b8fa82a0ac87a207d7afa2061d90a9bd
Parents: 022bf78
Author: Erik Krogen <ek...@linkedin.com>
Authored: Thu Oct 27 15:14:21 2016 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Thu Oct 27 15:58:25 2016 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/DatanodeManager.java | 16 ++++++---
 .../blockmanagement/TestDatanodeManager.java    | 37 ++++++++++++++++++++
 2 files changed, 49 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3ac1f41/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 1a47835..47f15c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -759,17 +759,25 @@ public class DatanodeManager {
     }
   }
 
+  /**
+   * Will return true for all Datanodes which have a non-null software
+   * version and are considered alive (by {@link DatanodeDescriptor#isAlive()}),
+   * indicating the node has not yet been removed. Use {@code isAlive}
+   * rather than {@link DatanodeManager#isDatanodeDead(DatanodeDescriptor)}
+   * to ensure that the version is decremented even if the datanode
+   * hasn't issued a heartbeat recently.
+   *
+   * @param node The datanode in question
+   * @return True iff its version count should be decremented
+   */
   private boolean shouldCountVersion(DatanodeDescriptor node) {
-    return node.getSoftwareVersion() != null && node.isAlive() &&
-      !isDatanodeDead(node);
+    return node.getSoftwareVersion() != null && node.isAlive();
   }
 
   private void countSoftwareVersions() {
     synchronized(this) {
       datanodesSoftwareVersions.clear();
       for(DatanodeDescriptor dn: datanodeMap.values()) {
-        // Check isAlive too because right after removeDatanode(),
-        // isDatanodeDead() is still true 
         if (shouldCountVersion(dn)) {
           Integer num = datanodesSoftwareVersions.get(dn.getSoftwareVersion());
           num = num == null ? 1 : num+1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3ac1f41/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index be8a0f0..30e2aaf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -86,6 +86,43 @@ public class TestDatanodeManager {
   }
 
   /**
+   * This test checks that if a node is re-registered with a new software
+   * version after the heartbeat expiry interval but before the HeartbeatManager
+   * has a chance to detect this and remove it, the node's version will still
+   * be correctly decremented.
+   */
+  @Test
+  public void testNumVersionsCorrectAfterReregister()
+      throws IOException, InterruptedException {
+    //Create the DatanodeManager which will be tested
+    FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
+    Mockito.when(fsn.hasWriteLock()).thenReturn(true);
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 0);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 10);
+    DatanodeManager dm = mockDatanodeManager(fsn, conf);
+
+    String storageID = "someStorageID1";
+    String ip = "someIP" + storageID;
+
+    // Register then reregister the same node but with a different version
+    for (int i = 0; i <= 1; i++) {
+      dm.registerDatanode(new DatanodeRegistration(
+          new DatanodeID(ip, "", storageID, 9000, 0, 0, 0),
+          null, null, "version" + i));
+      if (i == 0) {
+        Thread.sleep(25);
+      }
+    }
+
+    //Verify DatanodeManager has the correct count
+    Map<String, Integer> mapToCheck = dm.getDatanodesSoftwareVersions();
+    assertNull("should be no more version0 nodes", mapToCheck.get("version0"));
+    assertEquals("should be one version1 node",
+        mapToCheck.get("version1").intValue(), 1);
+  }
+
+  /**
    * This test sends a random sequence of node registrations and node removals
    * to the DatanodeManager (of nodes with different IDs and versions), and
    * checks that the DatanodeManager keeps a correct count of different software


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


[44/50] [abbrv] hadoop git commit: HDFS-11064. Mention the default NN rpc ports in hdfs-default.xml. Contributed by Yiqun Lin.

Posted by vi...@apache.org.
HDFS-11064. Mention the default NN rpc ports in hdfs-default.xml. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-9806
Commit: 57187fdb93c8f59372c980eb3d86073a3c8045b9
Parents: b62bc2b
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Oct 27 18:13:06 2016 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Oct 27 18:13:06 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml                | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/57187fdb/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 061d078..e28dc54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -37,7 +37,7 @@
     RPC address that handles all clients requests. In the case of HA/Federation where multiple namenodes exist,
     the name service id is added to the name e.g. dfs.namenode.rpc-address.ns1
     dfs.namenode.rpc-address.EXAMPLENAMESERVICE
-    The value of this property will take the form of nn-host1:rpc-port.
+    The value of this property will take the form of nn-host1:rpc-port. The NameNode's default RPC port is 9820.
   </description>
 </property>
 


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


[18/50] [abbrv] hadoop git commit: YARN-3432. Cluster metrics have wrong Total Memory when there is reserved memory on CS. (Brahma Reddy Battula via curino)

Posted by vi...@apache.org.
YARN-3432.  Cluster metrics have wrong Total Memory when there is reserved memory on CS. (Brahma Reddy Battula via curino)


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

Branch: refs/heads/HDFS-9806
Commit: 892a8348fceb42069ea9877251c413fe33415e16
Parents: 79ae78d
Author: Carlo Curino <cu...@apache.org>
Authored: Thu Oct 27 15:12:10 2016 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Oct 27 15:15:49 2016 -0700

----------------------------------------------------------------------
 .../resourcemanager/webapp/dao/ClusterMetricsInfo.java   | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/892a8348/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
index 1789e09..f083b05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 
 @XmlRootElement(name = "clusterMetrics")
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -87,8 +88,14 @@ public class ClusterMetricsInfo {
     this.containersPending = metrics.getPendingContainers();
     this.containersReserved = metrics.getReservedContainers();
 
-    this.totalMB = availableMB + allocatedMB;
-    this.totalVirtualCores = availableVirtualCores + allocatedVirtualCores;
+    if (rs instanceof CapacityScheduler) {
+      this.totalMB = availableMB + allocatedMB + reservedMB;
+      this.totalVirtualCores = availableVirtualCores + allocatedVirtualCores
+          + containersReserved;
+    } else {
+      this.totalMB = availableMB + allocatedMB;
+      this.totalVirtualCores = availableVirtualCores + allocatedVirtualCores;
+    }
     this.activeNodes = clusterMetrics.getNumActiveNMs();
     this.lostNodes = clusterMetrics.getNumLostNMs();
     this.unhealthyNodes = clusterMetrics.getUnhealthyNMs();


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


[13/50] [abbrv] hadoop git commit: YARN-4831. Recovered containers will be killed after NM stateful restart. Contributed by Siqi Li

Posted by vi...@apache.org.
YARN-4831. Recovered containers will be killed after NM stateful restart. Contributed by Siqi Li


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

Branch: refs/heads/HDFS-9806
Commit: 7e3c327d316b33d6a09bfd4e65e7e5384943bb1d
Parents: 31ff42b
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Oct 27 20:41:43 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Oct 27 20:42:52 2016 +0000

----------------------------------------------------------------------
 .../container/ContainerImpl.java                | 24 ++++++++++++--------
 1 file changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e3c327d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 4bc0a0f..e6b9d9f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -1506,16 +1506,20 @@ public class ContainerImpl implements Container {
   static class KillOnNewTransition extends ContainerDoneTransition {
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
-      ContainerKillEvent killEvent = (ContainerKillEvent) event;
-      container.exitCode = killEvent.getContainerExitStatus();
-      container.addDiagnostics(killEvent.getDiagnostic(), "\n");
-      container.addDiagnostics("Container is killed before being launched.\n");
-      container.metrics.killedContainer();
-      NMAuditLogger.logSuccess(container.user,
-          AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
-          container.containerId.getApplicationAttemptId().getApplicationId(),
-          container.containerId);
-      super.transition(container, event);
+      if (container.recoveredStatus == RecoveredContainerStatus.COMPLETED) {
+        container.sendFinishedEvents();
+      } else {
+        ContainerKillEvent killEvent = (ContainerKillEvent) event;
+        container.exitCode = killEvent.getContainerExitStatus();
+        container.addDiagnostics(killEvent.getDiagnostic(), "\n");
+        container.addDiagnostics("Container is killed before being launched.\n");
+        container.metrics.killedContainer();
+        NMAuditLogger.logSuccess(container.user,
+            AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
+            container.containerId.getApplicationAttemptId().getApplicationId(),
+            container.containerId);
+        super.transition(container, event);
+      }
     }
   }
 


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


[38/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
index d8755ec..271c339 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
@@ -101,7 +102,7 @@ public class KMS {
   @POST
   @Path(KMSRESTConstants.KEYS_RESOURCE)
   @Consumes(MediaType.APPLICATION_JSON)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   @SuppressWarnings("unchecked")
   public Response createKey(Map jsonKey) throws Exception {
     try{
@@ -204,7 +205,7 @@ public class KMS {
   @POST
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   @Consumes(MediaType.APPLICATION_JSON)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response rolloverKey(@PathParam("name") final String name,
       Map jsonMaterial) throws Exception {
     try {
@@ -254,7 +255,7 @@ public class KMS {
 
   @GET
   @Path(KMSRESTConstants.KEYS_METADATA_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getKeysMetadata(@QueryParam(KMSRESTConstants.KEY)
       List<String> keyNamesList) throws Exception {
     try {
@@ -287,7 +288,7 @@ public class KMS {
 
   @GET
   @Path(KMSRESTConstants.KEYS_NAMES_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getKeyNames() throws Exception {
     try {
       LOG.trace("Entering getKeyNames method.");
@@ -332,7 +333,7 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.METADATA_SUB_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getMetadata(@PathParam("name") final String name)
       throws Exception {
     try {
@@ -366,7 +367,7 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.CURRENT_VERSION_SUB_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getCurrentVersion(@PathParam("name") final String name)
       throws Exception {
     try {
@@ -399,7 +400,7 @@ public class KMS {
 
   @GET
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getKeyVersion(
       @PathParam("versionName") final String versionName) throws Exception {
     try {
@@ -436,7 +437,7 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.EEK_SUB_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response generateEncryptedKeys(
           @PathParam("name") final String name,
           @QueryParam(KMSRESTConstants.EEK_OP) String edekOp,
@@ -508,7 +509,7 @@ public class KMS {
   @POST
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}/" +
       KMSRESTConstants.EEK_SUB_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response decryptEncryptedKey(
       @PathParam("versionName") final String versionName,
       @QueryParam(KMSRESTConstants.EEK_OP) String eekOp,
@@ -577,7 +578,7 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.VERSIONS_SUB_RESOURCE)
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getKeyVersions(@PathParam("name") final String name)
       throws Exception {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
index 928a8aa..8efef73 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
@@ -114,7 +114,19 @@ public class KMSAuthenticationFilter
       super.sendError(sc);
     }
 
+    /**
+     * Calls setStatus(int sc, String msg) on the wrapped
+     * {@link HttpServletResponseWrapper} object.
+     *
+     * @param sc the status code
+     * @param sm the status message
+     * @deprecated {@link HttpServletResponseWrapper#setStatus(int, String)} is
+     * deprecated. To set a status code use {@link #setStatus(int)}, to send an
+     * error with a description use {@link #sendError(int, String)}
+     */
     @Override
+    @Deprecated
+    @SuppressWarnings("deprecation")
     public void setStatus(int sc, String sm) {
       statusCode = sc;
       msg = sm;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java
index 31fac9f..4b8a5be 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.crypto.key.kms.server;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.http.JettyUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import javax.ws.rs.Produces;
@@ -41,7 +42,7 @@ import java.util.Map;
  * to their JSON representation.
  */
 @Provider
-@Produces(MediaType.APPLICATION_JSON)
+@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
 @InterfaceAudience.Private
 public class KMSJSONWriter implements MessageBodyWriter<Object> {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java
index 292d731..8b181ad 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java
@@ -21,14 +21,19 @@ import com.google.common.base.Preconditions;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.util.ThreadUtil;
-import org.apache.hadoop.security.ssl.SslSelectChannelConnectorSecure;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.security.SslSelectChannelConnector;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.ConnectionFactory;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -37,11 +42,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Writer;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.MalformedURLException;
-import java.net.ServerSocket;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.UUID;
 
@@ -51,18 +52,28 @@ public class MiniKMS {
     try {
       boolean ssl = keyStore != null;
       String host = "localhost";
-      Server server = new Server(inPort);
-      if (!ssl) {
-        server.getConnectors()[0].setHost(host);
-      } else {
-        SslSelectChannelConnector c = new SslSelectChannelConnectorSecure();
-        c.setHost(host);
-        c.setNeedClientAuth(false);
-        c.setKeystore(keyStore);
-        c.setKeystoreType("jks");
-        c.setKeyPassword(password);
-        server.setConnectors(new Connector[]{c});
+      Server server = new Server();
+      ServerConnector conn = new ServerConnector(server);
+      HttpConfiguration httpConfig = new HttpConfiguration();
+      httpConfig.setRequestHeaderSize(JettyUtils.HEADER_SIZE);
+      httpConfig.setResponseHeaderSize(JettyUtils.HEADER_SIZE);
+      httpConfig.setSecureScheme("https");
+      httpConfig.addCustomizer(new SecureRequestCustomizer());
+      ConnectionFactory connFactory = new HttpConnectionFactory(httpConfig);
+      conn.addConnectionFactory(connFactory);
+      conn.setHost(host);
+      conn.setPort(inPort);
+      if (ssl) {
+        SslContextFactory sslContextFactory = new SslContextFactory();
+        sslContextFactory.setNeedClientAuth(false);
+        sslContextFactory.setKeyStorePath(keyStore);
+        sslContextFactory.setKeyStoreType("jks");
+        sslContextFactory.setKeyStorePassword(password);
+        conn.addFirstConnectionFactory(
+            new SslConnectionFactory(sslContextFactory,
+            HttpVersion.HTTP_1_1.asString()));
       }
+      server.addConnector(conn);
       return server;
     } catch (Exception ex) {
       throw new RuntimeException("Could not start embedded servlet container, "
@@ -71,13 +82,13 @@ public class MiniKMS {
   }
 
   private static URL getJettyURL(Server server) {
-    boolean ssl = server.getConnectors()[0].getClass()
-        == SslSelectChannelConnectorSecure.class;
+    boolean ssl = server.getConnectors()[0]
+        .getConnectionFactory(SslConnectionFactory.class) != null;
     try {
       String scheme = (ssl) ? "https" : "http";
       return new URL(scheme + "://" +
-          server.getConnectors()[0].getHost() + ":" +
-          server.getConnectors()[0].getLocalPort());
+          ((ServerConnector)server.getConnectors()[0]).getHost() + ":"
+          + ((ServerConnector)server.getConnectors()[0]).getLocalPort());
     } catch (MalformedURLException ex) {
       throw new RuntimeException("It should never happen, " + ex.getMessage(),
           ex);
@@ -217,7 +228,7 @@ public class MiniKMS {
     if (webXmlInJar) {
       context.setClassLoader(cl);
     }
-    jetty.addHandler(context);
+    jetty.setHandler(context);
     jetty.start();
     kmsURL = new URL(getJettyURL(jetty), "kms");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-common-project/hadoop-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml
index c943f3c..027cf34 100644
--- a/hadoop-common-project/hadoop-nfs/pom.xml
+++ b/hadoop-common-project/hadoop-nfs/pom.xml
@@ -63,7 +63,7 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
index 0aa5fc1..928ada9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
@@ -75,7 +75,7 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>
@@ -89,8 +89,8 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -108,22 +108,22 @@
         </exclusion>
         <exclusion>
           <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
         <exclusion>
           <groupId>javax.servlet.jsp</groupId>
           <artifactId>jsp-api</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>jetty-util</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>servlet-api-2.5</artifactId>
         </exclusion>
         <exclusion>
@@ -155,18 +155,18 @@
         </exclusion>
         <exclusion>
           <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>jetty-util</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
+          <groupId>org.eclipse.jetty</groupId>
           <artifactId>servlet-api-2.5</artifactId>
         </exclusion>
         <exclusion>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index a4db124..677bca7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrEncodingPa
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrSetFlagParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrValueParam;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.service.FileSystemAccessException;
 import org.apache.hadoop.lib.service.Groups;
@@ -168,7 +169,7 @@ public class HttpFSServer {
    * {@link HttpFSExceptionProvider}.
    */
   @GET
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response getRoot(@QueryParam(OperationParam.NAME) OperationParam op,
                           @Context Parameters params,
                           @Context HttpServletRequest request)
@@ -197,7 +198,8 @@ public class HttpFSServer {
    */
   @GET
   @Path("{path:.*}")
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response get(@PathParam("path") String path,
                       @QueryParam(OperationParam.NAME) OperationParam op,
                       @Context Parameters params,
@@ -363,7 +365,7 @@ public class HttpFSServer {
    */
   @DELETE
   @Path("{path:.*}")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response delete(@PathParam("path") String path,
                          @QueryParam(OperationParam.NAME) OperationParam op,
                          @Context Parameters params,
@@ -414,7 +416,7 @@ public class HttpFSServer {
   @POST
   @Path("{path:.*}")
   @Consumes({"*/*"})
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response post(InputStream is,
                        @Context UriInfo uriInfo,
                        @PathParam("path") String path,
@@ -509,7 +511,7 @@ public class HttpFSServer {
   @PUT
   @Path("{path:.*}")
   @Consumes({"*/*"})
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response put(InputStream is,
                        @Context UriInfo uriInfo,
                        @PathParam("path") String path,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java
index 05bb9a1..a1525a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.lib.wsrs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.http.JettyUtils;
 import org.json.simple.JSONObject;
 
 import javax.ws.rs.Produces;
@@ -37,7 +38,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.Map;
 
 @Provider
-@Produces(MediaType.APPLICATION_JSON)
+@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
 @InterfaceAudience.Private
 public class JSONMapProvider implements MessageBodyWriter<Map> {
   private static final String ENTER = System.getProperty("line.separator");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java
index 7e1f98b..8ae9145 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.lib.wsrs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.http.JettyUtils;
 import org.json.simple.JSONStreamAware;
 
 import javax.ws.rs.Produces;
@@ -36,7 +37,7 @@ import java.lang.reflect.Type;
 import java.nio.charset.StandardCharsets;
 
 @Provider
-@Produces(MediaType.APPLICATION_JSON)
+@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
 @InterfaceAudience.Private
 public class JSONProvider implements MessageBodyWriter<JSONStreamAware> {
   private static final String ENTER = System.getProperty("line.separator");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index e475803..4e25e56 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -49,8 +49,8 @@ import org.junit.Assume;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.collect.Lists;
 
@@ -127,7 +127,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     URL url = cl.getResource("webapp");
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
index c6a7a9d..a9c36b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
@@ -64,8 +64,8 @@ import org.apache.hadoop.test.TestJettyHelper;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.collect.Maps;
 import java.util.Properties;
@@ -171,7 +171,7 @@ public class TestHttpFSServer extends HFSTestCase {
     URL url = cl.getResource("webapp");
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
     if (addDelegationTokenAuthHandler) {
       HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
index cadec2e..289ddc4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.test.TestJetty;
 import org.apache.hadoop.test.TestJettyHelper;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -151,7 +151,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
     }
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
index 951b3fb..7571125 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
@@ -32,8 +32,8 @@ import org.apache.hadoop.test.TestJetty;
 import org.apache.hadoop.test.TestJettyHelper;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -152,7 +152,7 @@ public class TestHttpFSServerNoXAttrs extends HTestCase {
     }
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
index 757e3fd..fafeff0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
@@ -41,8 +41,8 @@ import org.json.simple.parser.JSONParser;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.webapp.WebAppContext;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.webapp.WebAppContext;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -105,7 +105,7 @@ public class TestHttpFSWithKerberos extends HFSTestCase {
     URL url = cl.getResource("webapp");
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
     HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java
index eb2cdc6..2d09b80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java
@@ -38,9 +38,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.Time;
+import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
+import org.eclipse.jetty.server.Server;
 
 public class TestHFSTestCase extends HFSTestCase {
 
@@ -165,11 +165,11 @@ public class TestHFSTestCase extends HFSTestCase {
   @Test
   @TestJetty
   public void testJetty() throws Exception {
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/");
     context.addServlet(MyServlet.class, "/bar");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
     URL url = new URL(TestJettyHelper.getJettyURL(), "/bar");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java
index 74d34ec..be01285 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java
@@ -33,9 +33,9 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.util.Time;
+import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.junit.Test;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
+import org.eclipse.jetty.server.Server;
 
 public class TestHTestCase extends HTestCase {
 
@@ -132,11 +132,11 @@ public class TestHTestCase extends HTestCase {
   @Test
   @TestJetty
   public void testJetty() throws Exception {
-    Context context = new Context();
+    ServletContextHandler context = new ServletContextHandler();
     context.setContextPath("/");
     context.addServlet(MyServlet.class, "/bar");
     Server server = TestJettyHelper.getJettyServer();
-    server.addHandler(context);
+    server.setHandler(context);
     server.start();
     URL url = new URL(TestJettyHelper.getJettyURL(), "/bar");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
index a9b661e..1da3901 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
@@ -24,14 +24,19 @@ import java.net.ServerSocket;
 import java.net.URL;
 import java.net.UnknownHostException;
 
-import org.apache.hadoop.security.ssl.SslSelectChannelConnectorSecure;
-import org.junit.Test;
+import org.apache.hadoop.http.JettyUtils;
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.ConnectionFactory;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.junit.rules.MethodRule;
 import org.junit.runners.model.FrameworkMethod;
 import org.junit.runners.model.Statement;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.security.SslSelectChannelConnector;
 
 public class TestJettyHelper implements MethodRule {
   private boolean ssl;
@@ -88,20 +93,27 @@ public class TestJettyHelper implements MethodRule {
       ServerSocket ss = new ServerSocket(0, 50, localhost);
       int port = ss.getLocalPort();
       ss.close();
-      Server server = new Server(0);
-      if (!ssl) {
-        server.getConnectors()[0].setHost(host);
-        server.getConnectors()[0].setPort(port);
-      } else {
-        SslSelectChannelConnector c = new SslSelectChannelConnectorSecure();
-        c.setHost(host);
-        c.setPort(port);
-        c.setNeedClientAuth(false);
-        c.setKeystore(keyStore);
-        c.setKeystoreType(keyStoreType);
-        c.setKeyPassword(keyStorePassword);
-        server.setConnectors(new Connector[] {c});
+      Server server = new Server();
+      ServerConnector conn = new ServerConnector(server);
+      HttpConfiguration http_config = new HttpConfiguration();
+      http_config.setRequestHeaderSize(JettyUtils.HEADER_SIZE);
+      http_config.setResponseHeaderSize(JettyUtils.HEADER_SIZE);
+      http_config.setSecureScheme("https");
+      http_config.addCustomizer(new SecureRequestCustomizer());
+      ConnectionFactory connFactory = new HttpConnectionFactory(http_config);
+      conn.addConnectionFactory(connFactory);
+      conn.setHost(host);
+      conn.setPort(port);
+      if (ssl) {
+        SslContextFactory sslContextFactory = new SslContextFactory();
+        sslContextFactory.setNeedClientAuth(false);
+        sslContextFactory.setKeyStorePath(keyStore);
+        sslContextFactory.setKeyStoreType(keyStoreType);
+        sslContextFactory.setKeyStorePassword(keyStorePassword);
+        conn.addFirstConnectionFactory(new SslConnectionFactory(sslContextFactory,
+            HttpVersion.HTTP_1_1.asString()));
       }
+      server.addConnector(conn);
       return server;
     } catch (Exception ex) {
       throw new RuntimeException("Could not start embedded servlet container, " + ex.getMessage(), ex);
@@ -117,8 +129,8 @@ public class TestJettyHelper implements MethodRule {
     Server server = getJettyServer();
     try {
       InetAddress add =
-        InetAddress.getByName(server.getConnectors()[0].getHost());
-      int port = server.getConnectors()[0].getPort();
+        InetAddress.getByName(((ServerConnector)server.getConnectors()[0]).getHost());
+      int port = ((ServerConnector)server.getConnectors()[0]).getPort();
       return new InetSocketAddress(add, port);
     } catch (UnknownHostException ex) {
       throw new RuntimeException(ex);
@@ -157,8 +169,8 @@ public class TestJettyHelper implements MethodRule {
     try {
       String scheme = (helper.ssl) ? "https" : "http";
       return new URL(scheme + "://" +
-          helper.server.getConnectors()[0].getHost() + ":" +
-          helper.server.getConnectors()[0].getPort());
+          ((ServerConnector)helper.server.getConnectors()[0]).getHost() + ":" +
+          ((ServerConnector)helper.server.getConnectors()[0]).getPort());
     } catch (MalformedURLException ex) {
       throw new RuntimeException("It should never happen, " + ex.getMessage(), ex);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
index 7bbfe87..5746814 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
@@ -85,12 +85,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
       <scope>compile</scope>
     </dependency>
@@ -146,7 +146,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index b06cd4c..36db4d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -75,16 +75,21 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <groupId>com.sun.jersey</groupId>
       <artifactId>jersey-core</artifactId>
       <scope>compile</scope>
@@ -136,7 +141,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
@@ -356,6 +361,18 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
+          <execution>
+            <id>resource-gz</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>resource-gz</goal>
+            </goals>
+            <configuration>
+              <inputDirectory>${basedir}/src/main/webapps/static</inputDirectory>
+              <outputDirectory>${basedir}/target/webapps/static</outputDirectory>
+              <extensions>js,css</extensions>
+            </configuration>
+          </execution>
         </executions>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
index d08b40f..cde0112 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.htrace.core.Tracer;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 9ceffc2..f89d38c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -212,7 +212,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.htrace.core.Tracer;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 88c7681..3c782e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -283,7 +283,7 @@ import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Appender;
 import org.apache.log4j.AsyncAppender;
 import org.apache.log4j.Logger;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index 5135838..a846f60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.hdfs.util.PersistentLongFile;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.util.Time;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 0186d8b..e4b95ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -64,7 +64,7 @@ import org.apache.http.client.utils.URIBuilder;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
-import org.mortbay.jetty.EofException;
+import org.eclipse.jetty.io.EofException;
 
 /**
  * This class provides fetching a specified file from the NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 4247a67..cea3339 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -84,6 +84,7 @@ import org.apache.hadoop.hdfs.web.ParamFilter;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.*;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ExternalCall;
 import org.apache.hadoop.ipc.RetriableException;
@@ -355,7 +356,8 @@ public class NamenodeWebHdfsMethods {
   @PUT
   @Path("/")
   @Consumes({"*/*"})
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response putRoot(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -395,13 +397,13 @@ public class NamenodeWebHdfsMethods {
           final CreateParentParam createParent,
       @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
           final TokenArgumentParam delegationTokenArgument,
-      @QueryParam(AclPermissionParam.NAME) @DefaultValue(AclPermissionParam.DEFAULT) 
+      @QueryParam(AclPermissionParam.NAME) @DefaultValue(AclPermissionParam.DEFAULT)
           final AclPermissionParam aclPermission,
-      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT) 
+      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
           final XAttrNameParam xattrName,
-      @QueryParam(XAttrValueParam.NAME) @DefaultValue(XAttrValueParam.DEFAULT) 
+      @QueryParam(XAttrValueParam.NAME) @DefaultValue(XAttrValueParam.DEFAULT)
           final XAttrValueParam xattrValue,
-      @QueryParam(XAttrSetFlagParam.NAME) @DefaultValue(XAttrSetFlagParam.DEFAULT) 
+      @QueryParam(XAttrSetFlagParam.NAME) @DefaultValue(XAttrSetFlagParam.DEFAULT)
           final XAttrSetFlagParam xattrSetFlag,
       @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
           final SnapshotNameParam snapshotName,
@@ -426,7 +428,8 @@ public class NamenodeWebHdfsMethods {
   @PUT
   @Path("{" + UriFsPathParam.NAME + ":.*}")
   @Consumes({"*/*"})
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response put(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -532,7 +535,7 @@ public class NamenodeWebHdfsMethods {
       final TokenArgumentParam delegationTokenArgument,
       final AclPermissionParam aclPermission,
       final XAttrNameParam xattrName,
-      final XAttrValueParam xattrValue, 
+      final XAttrValueParam xattrValue,
       final XAttrSetFlagParam xattrSetFlag,
       final SnapshotNameParam snapshotName,
       final OldSnapshotNameParam oldSnapshotName,
@@ -690,7 +693,8 @@ public class NamenodeWebHdfsMethods {
   @POST
   @Path("/")
   @Consumes({"*/*"})
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response postRoot(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -720,7 +724,8 @@ public class NamenodeWebHdfsMethods {
   @POST
   @Path("{" + UriFsPathParam.NAME + ":.*}")
   @Consumes({"*/*"})
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response post(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -799,7 +804,7 @@ public class NamenodeWebHdfsMethods {
             "newLength parameter is Missing");
       }
       // We treat each rest request as a separate client.
-      final boolean b = np.truncate(fullpath, newLength.getValue(), 
+      final boolean b = np.truncate(fullpath, newLength.getValue(),
           "DFSClient_" + DFSUtil.getSecureRandom().nextLong());
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
@@ -812,7 +817,8 @@ public class NamenodeWebHdfsMethods {
   /** Handle HTTP GET request for the root. */
   @GET
   @Path("/")
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response getRoot(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -831,9 +837,9 @@ public class NamenodeWebHdfsMethods {
           final RenewerParam renewer,
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
           final BufferSizeParam bufferSize,
-      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT) 
+      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
           final List<XAttrNameParam> xattrNames,
-      @QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT) 
+      @QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT)
           final XAttrEncodingParam xattrEncoding,
       @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
           final ExcludeDatanodesParam excludeDatanodes,
@@ -856,7 +862,8 @@ public class NamenodeWebHdfsMethods {
   /** Handle HTTP GET request. */
   @GET
   @Path("{" + UriFsPathParam.NAME + ":.*}")
-  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_OCTET_STREAM + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8})
   public Response get(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -1131,7 +1138,7 @@ public class NamenodeWebHdfsMethods {
   /** Handle HTTP DELETE request for the root. */
   @DELETE
   @Path("/")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response deleteRoot(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
@@ -1154,7 +1161,7 @@ public class NamenodeWebHdfsMethods {
   /** Handle HTTP DELETE request. */
   @DELETE
   @Path("{" + UriFsPathParam.NAME + ":.*}")
-  @Produces(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
   public Response delete(
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 6ca1e79..34ae12c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -64,7 +64,7 @@ import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
index 45b245a..1de37a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * Test {@link JournalNodeMXBean}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
index 9e3112e..476565dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * Class for testing {@link BlockStatsMXBean} implementation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
index a77c943..6933c3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
@@ -40,7 +40,7 @@ import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
index 3c0d54a..4687cfd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * Class for testing {@link NameNodeMXBean} implementation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index 47f1c85..659a1d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.util.VersionInfo;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java
index 0f22e9a..bff549a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java
@@ -36,7 +36,7 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 public class TestStartupProgressServlet {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
index ca97e32..df02b35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
@@ -120,7 +120,7 @@ public class TestTransferFsImage {
   /**
    * Test to verify the read timeout
    */
-  @Test(timeout = 5000)
+  @Test(timeout = 10000)
   public void testGetImageTimeout() throws Exception {
     HttpServer2 testServer = HttpServerFunctionalTest.createServer("hdfs");
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java
index 9f78548..4e1ceed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.core.Response;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
index f8e931a..1373891 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * This class drives the creation of a mini-cluster on the local machine. By

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/JobEndNotifier.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/JobEndNotifier.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/JobEndNotifier.java
index 05bb40b..836fad5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/JobEndNotifier.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/JobEndNotifier.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 /**
  * <p>This class handles job end notification. Submitters of jobs can choose to
@@ -101,11 +101,12 @@ public class JobEndNotifier implements Configurable {
         int port = Integer.parseInt(portConf);
         proxyToUse = new Proxy(proxyType,
           new InetSocketAddress(hostname, port));
-        Log.info("Job end notification using proxy type \"" + proxyType + 
-        "\" hostname \"" + hostname + "\" and port \"" + port + "\"");
+        Log.getLog().info("Job end notification using proxy type \""
+            + proxyType + "\" hostname \"" + hostname + "\" and port \"" + port
+            + "\"");
       } catch(NumberFormatException nfe) {
-        Log.warn("Job end notification couldn't parse configured proxy's port "
-          + portConf + ". Not going to use a proxy");
+        Log.getLog().warn("Job end notification couldn't parse configured"
+            + "proxy's port " + portConf + ". Not going to use a proxy");
       }
     }
 
@@ -121,23 +122,25 @@ public class JobEndNotifier implements Configurable {
   protected boolean notifyURLOnce() {
     boolean success = false;
     try {
-      Log.info("Job end notification trying " + urlToNotify);
+      Log.getLog().info("Job end notification trying " + urlToNotify);
       HttpURLConnection conn =
         (HttpURLConnection) urlToNotify.openConnection(proxyToUse);
       conn.setConnectTimeout(timeout);
       conn.setReadTimeout(timeout);
       conn.setAllowUserInteraction(false);
       if(conn.getResponseCode() != HttpURLConnection.HTTP_OK) {
-        Log.warn("Job end notification to " + urlToNotify +" failed with code: "
-        + conn.getResponseCode() + " and message \"" + conn.getResponseMessage()
-        +"\"");
+        Log.getLog().warn("Job end notification to " + urlToNotify
+            + " failed with code: " + conn.getResponseCode() + " and message \""
+            + conn.getResponseMessage() + "\"");
       }
       else {
         success = true;
-        Log.info("Job end notification to " + urlToNotify + " succeeded");
+        Log.getLog().info("Job end notification to " + urlToNotify
+            + " succeeded");
       }
     } catch(IOException ioe) {
-      Log.warn("Job end notification to " + urlToNotify + " failed", ioe);
+      Log.getLog().warn("Job end notification to " + urlToNotify + " failed",
+          ioe);
     }
     return success;
   }
@@ -152,7 +155,7 @@ public class JobEndNotifier implements Configurable {
     throws InterruptedException {
     // Do we need job-end notification?
     if (userUrl == null) {
-      Log.info("Job end notification URL not set, skipping.");
+      Log.getLog().info("Job end notification URL not set, skipping.");
       return;
     }
 
@@ -168,23 +171,25 @@ public class JobEndNotifier implements Configurable {
     try {
       urlToNotify = new URL(userUrl);
     } catch (MalformedURLException mue) {
-      Log.warn("Job end notification couldn't parse " + userUrl, mue);
+      Log.getLog().warn("Job end notification couldn't parse " + userUrl, mue);
       return;
     }
 
     // Send notification
     boolean success = false;
     while (numTries-- > 0 && !success) {
-      Log.info("Job end notification attempts left " + numTries);
+      Log.getLog().info("Job end notification attempts left " + numTries);
       success = notifyURLOnce();
       if (!success) {
         Thread.sleep(waitInterval);
       }
     }
     if (!success) {
-      Log.warn("Job end notification failed to notify : " + urlToNotify);
+      Log.getLog().warn("Job end notification failed to notify : "
+          + urlToNotify);
     } else {
-      Log.info("Job end notification succeeded for " + jobReport.getJobId());
+      Log.getLog().info("Job end notification succeeded for "
+          + jobReport.getJobId());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
index 5d50db7..f477d31 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
@@ -38,6 +38,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
 
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptResponse;
@@ -221,14 +222,16 @@ public class AMWebServices {
   }
 
   @GET
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppInfo get() {
     return getAppInfo();
   }
 
   @GET
   @Path("/info")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppInfo getAppInfo() {
     init();
     return new AppInfo(this.app, this.app.context);
@@ -236,7 +239,8 @@ public class AMWebServices {
   
   @GET
   @Path("/blacklistednodes")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public BlacklistedNodesInfo getBlacklistedNodes() {
     init();
     return new BlacklistedNodesInfo(this.app.context);
@@ -244,7 +248,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobsInfo getJobs(@Context HttpServletRequest hsr) {
     init();
     JobsInfo allJobs = new JobsInfo();
@@ -261,7 +266,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobInfo getJob(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid) {
     init();
@@ -271,7 +277,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/jobattempts")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AMAttemptsInfo getJobAttempts(@PathParam("jobid") String jid) {
     init();
     Job job = getJobFromJobIdString(jid, appCtx);
@@ -286,7 +293,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/counters")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobCounterInfo getJobCounters(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid) {
     init();
@@ -297,7 +305,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/conf")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public ConfInfo getJobConf(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid) {
 
@@ -316,7 +325,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TasksInfo getJobTasks(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @QueryParam("type") String type) {
 
@@ -343,7 +353,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks/{taskid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TaskInfo getJobTask(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
 
@@ -356,7 +367,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks/{taskid}/counters")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobTaskCounterInfo getSingleTaskCounters(
       @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
       @PathParam("taskid") String tid) {
@@ -370,7 +382,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks/{taskid}/attempts")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TaskAttemptsInfo getJobTaskAttempts(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
 
@@ -394,7 +407,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public TaskAttemptInfo getJobTaskAttemptId(@Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid,
       @PathParam("attemptid") String attId) {
@@ -413,7 +427,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/state")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobTaskAttemptState getJobTaskAttemptState(
       @Context HttpServletRequest hsr,
       @PathParam("jobid") String jid, @PathParam("taskid") String tid,
@@ -429,7 +444,8 @@ public class AMWebServices {
 
   @PUT
   @Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/state")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public Response updateJobTaskAttemptState(JobTaskAttemptState targetState,
       @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
@@ -466,7 +482,8 @@ public class AMWebServices {
 
   @GET
   @Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
       @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
       @PathParam("taskid") String tid, @PathParam("attemptid") String attId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 7138196..c5dc290 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
@@ -31,6 +31,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
@@ -112,7 +113,8 @@ public class TestAMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyAMInfo(json.getJSONObject("info"), appContext);
@@ -123,7 +125,8 @@ public class TestAMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("mapreduce/")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyAMInfo(json.getJSONObject("info"), appContext);
@@ -134,7 +137,8 @@ public class TestAMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("mapreduce/")
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyAMInfo(json.getJSONObject("info"), appContext);
@@ -145,7 +149,8 @@ public class TestAMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyAMInfoXML(xml, appContext);
   }
@@ -156,7 +161,8 @@ public class TestAMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("info").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyAMInfo(json.getJSONObject("info"), appContext);
@@ -168,7 +174,8 @@ public class TestAMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("info/").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyAMInfo(json.getJSONObject("info"), appContext);
@@ -179,7 +186,8 @@ public class TestAMWebServices extends JerseyTestBase {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("info/").get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyAMInfo(json.getJSONObject("info"), appContext);
@@ -191,7 +199,8 @@ public class TestAMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("info/").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyAMInfoXML(xml, appContext);
   }
@@ -251,7 +260,8 @@ public class TestAMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("blacklistednodes").accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     verifyBlacklistedNodesInfo(json, appContext);
@@ -263,7 +273,8 @@ public class TestAMWebServices extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("mapreduce")
         .path("blacklistednodes").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     String xml = response.getEntity(String.class);
     verifyBlacklistedNodesInfoXML(xml, appContext);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 f2e6d63..f20ac6f 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
@@ -32,6 +32,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
@@ -152,7 +153,8 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
               .path("attempts").path(attid).path("state")
               .queryParam("user.name", webserviceUserName)
               .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           assertEquals(att.getState().toString(), json.get("state"));
@@ -180,7 +182,8 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
               .queryParam("user.name", webserviceUserName)
               .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-          assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+              response.getType().toString());
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
           DocumentBuilder db = dbf.newDocumentBuilder();
@@ -219,7 +222,8 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
               .accept(MediaType.APPLICATION_JSON)
               .type(MediaType.APPLICATION_JSON)
               .put(ClientResponse.class, "{\"state\":\"KILLED\"}");
-          assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+                  + JettyUtils.UTF_8, response.getType().toString());
           JSONObject json = response.getEntity(JSONObject.class);
           assertEquals("incorrect number of elements", 1, json.length());
           assertEquals(TaskAttemptState.KILLED.toString(), json.get("state"));
@@ -252,7 +256,8 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
               .put(ClientResponse.class,
                   "<jobTaskAttemptState><state>KILLED" +
                       "</state></jobTaskAttemptState>");
-          assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+          assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+              response.getType().toString());
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
           DocumentBuilder db = dbf.newDocumentBuilder();


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


[34/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js
new file mode 100644
index 0000000..bc3fbc8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js
@@ -0,0 +1,2 @@
+/*! jQuery v1.8.2 jquery.com | jquery.org/license */
+(function(a,b){function G(a){var b=F[a]={};return p.each(a.split(s),function(a,c){b[c]=!0}),b}function J(a,c,d){if(d===b&&a.nodeType===1){var e="data-"+c.replace(I,"-$1").toLowerCase();d=a.getAttribute(e);if(typeof d=="string"){try{d=d==="true"?!0:d==="false"?!1:d==="null"?null:+d+""===d?+d:H.test(d)?p.parseJSON(d):d}catch(f){}p.data(a,c,d)}else d=b}return d}function K(a){var b;for(b in a){if(b==="data"&&p.isEmptyObject(a[b]))continue;if(b!=="toJSON")return!1}return!0}function ba(){return!1}function bb(){return!0}function bh(a){return!a||!a.parentNode||a.parentNode.nodeType===11}function bi(a,b){do a=a[b];while(a&&a.nodeType!==1);return a}function bj(a,b,c){b=b||0;if(p.isFunction(b))return p.grep(a,function(a,d){var e=!!b.call(a,d,a);return e===c});if(b.nodeType)return p.grep(a,function(a,d){return a===b===c});if(typeof b=="string"){var d=p.grep(a,function(a){return a.nodeType===1});if(be.test(b))return p.filter(b,d,!c);b=p.filter(b,d)}return p.grep(a,function(a,d){return p.inArray(
 a,b)>=0===c})}function bk(a){var b=bl.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}function bC(a,b){return a.getElementsByTagName(b)[0]||a.appendChild(a.ownerDocument.createElement(b))}function bD(a,b){if(b.nodeType!==1||!p.hasData(a))return;var c,d,e,f=p._data(a),g=p._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;d<e;d++)p.event.add(b,c,h[c][d])}g.data&&(g.data=p.extend({},g.data))}function bE(a,b){var c;if(b.nodeType!==1)return;b.clearAttributes&&b.clearAttributes(),b.mergeAttributes&&b.mergeAttributes(a),c=b.nodeName.toLowerCase(),c==="object"?(b.parentNode&&(b.outerHTML=a.outerHTML),p.support.html5Clone&&a.innerHTML&&!p.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):c==="input"&&bv.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):c==="option"?b.selected=a.defaultSelected:c==="input"||c==="textarea"?b.defaultValue=a.defaultValue:c==="scri
 pt"&&b.text!==a.text&&(b.text=a.text),b.removeAttribute(p.expando)}function bF(a){return typeof a.getElementsByTagName!="undefined"?a.getElementsByTagName("*"):typeof a.querySelectorAll!="undefined"?a.querySelectorAll("*"):[]}function bG(a){bv.test(a.type)&&(a.defaultChecked=a.checked)}function bY(a,b){if(b in a)return b;var c=b.charAt(0).toUpperCase()+b.slice(1),d=b,e=bW.length;while(e--){b=bW[e]+c;if(b in a)return b}return d}function bZ(a,b){return a=b||a,p.css(a,"display")==="none"||!p.contains(a.ownerDocument,a)}function b$(a,b){var c,d,e=[],f=0,g=a.length;for(;f<g;f++){c=a[f];if(!c.style)continue;e[f]=p._data(c,"olddisplay"),b?(!e[f]&&c.style.display==="none"&&(c.style.display=""),c.style.display===""&&bZ(c)&&(e[f]=p._data(c,"olddisplay",cc(c.nodeName)))):(d=bH(c,"display"),!e[f]&&d!=="none"&&p._data(c,"olddisplay",d))}for(f=0;f<g;f++){c=a[f];if(!c.style)continue;if(!b||c.style.display==="none"||c.style.display==="")c.style.display=b?e[f]||"":"none"}return a}function b_(a,b,c){
 var d=bP.exec(b);return d?Math.max(0,d[1]-(c||0))+(d[2]||"px"):b}function ca(a,b,c,d){var e=c===(d?"border":"content")?4:b==="width"?1:0,f=0;for(;e<4;e+=2)c==="margin"&&(f+=p.css(a,c+bV[e],!0)),d?(c==="content"&&(f-=parseFloat(bH(a,"padding"+bV[e]))||0),c!=="margin"&&(f-=parseFloat(bH(a,"border"+bV[e]+"Width"))||0)):(f+=parseFloat(bH(a,"padding"+bV[e]))||0,c!=="padding"&&(f+=parseFloat(bH(a,"border"+bV[e]+"Width"))||0));return f}function cb(a,b,c){var d=b==="width"?a.offsetWidth:a.offsetHeight,e=!0,f=p.support.boxSizing&&p.css(a,"boxSizing")==="border-box";if(d<=0||d==null){d=bH(a,b);if(d<0||d==null)d=a.style[b];if(bQ.test(d))return d;e=f&&(p.support.boxSizingReliable||d===a.style[b]),d=parseFloat(d)||0}return d+ca(a,b,c||(f?"border":"content"),e)+"px"}function cc(a){if(bS[a])return bS[a];var b=p("<"+a+">").appendTo(e.body),c=b.css("display");b.remove();if(c==="none"||c===""){bI=e.body.appendChild(bI||p.extend(e.createElement("iframe"),{frameBorder:0,width:0,height:0}));if(!bJ||!bI.
 createElement)bJ=(bI.contentWindow||bI.contentDocument).document,bJ.write("<!doctype html><html><body>"),bJ.close();b=bJ.body.appendChild(bJ.createElement(a)),c=bH(b,"display"),e.body.removeChild(bI)}return bS[a]=c,c}function ci(a,b,c,d){var e;if(p.isArray(b))p.each(b,function(b,e){c||ce.test(a)?d(a,e):ci(a+"["+(typeof e=="object"?b:"")+"]",e,c,d)});else if(!c&&p.type(b)==="object")for(e in b)ci(a+"["+e+"]",b[e],c,d);else d(a,b)}function cz(a){return function(b,c){typeof b!="string"&&(c=b,b="*");var d,e,f,g=b.toLowerCase().split(s),h=0,i=g.length;if(p.isFunction(c))for(;h<i;h++)d=g[h],f=/^\+/.test(d),f&&(d=d.substr(1)||"*"),e=a[d]=a[d]||[],e[f?"unshift":"push"](c)}}function cA(a,c,d,e,f,g){f=f||c.dataTypes[0],g=g||{},g[f]=!0;var h,i=a[f],j=0,k=i?i.length:0,l=a===cv;for(;j<k&&(l||!h);j++)h=i[j](c,d,e),typeof h=="string"&&(!l||g[h]?h=b:(c.dataTypes.unshift(h),h=cA(a,c,d,e,h,g)));return(l||!h)&&!g["*"]&&(h=cA(a,c,d,e,"*",g)),h}function cB(a,c){var d,e,f=p.ajaxSettings.flatOptions||{};f
 or(d in c)c[d]!==b&&((f[d]?a:e||(e={}))[d]=c[d]);e&&p.extend(!0,a,e)}function cC(a,c,d){var e,f,g,h,i=a.contents,j=a.dataTypes,k=a.responseFields;for(f in k)f in d&&(c[k[f]]=d[f]);while(j[0]==="*")j.shift(),e===b&&(e=a.mimeType||c.getResponseHeader("content-type"));if(e)for(f in i)if(i[f]&&i[f].test(e)){j.unshift(f);break}if(j[0]in d)g=j[0];else{for(f in d){if(!j[0]||a.converters[f+" "+j[0]]){g=f;break}h||(h=f)}g=g||h}if(g)return g!==j[0]&&j.unshift(g),d[g]}function cD(a,b){var c,d,e,f,g=a.dataTypes.slice(),h=g[0],i={},j=0;a.dataFilter&&(b=a.dataFilter(b,a.dataType));if(g[1])for(c in a.converters)i[c.toLowerCase()]=a.converters[c];for(;e=g[++j];)if(e!=="*"){if(h!=="*"&&h!==e){c=i[h+" "+e]||i["* "+e];if(!c)for(d in i){f=d.split(" ");if(f[1]===e){c=i[h+" "+f[0]]||i["* "+f[0]];if(c){c===!0?c=i[d]:i[d]!==!0&&(e=f[0],g.splice(j--,0,e));break}}}if(c!==!0)if(c&&a["throws"])b=c(b);else try{b=c(b)}catch(k){return{state:"parsererror",error:c?k:"No conversion from "+h+" to "+e}}}h=e}return{sta
 te:"success",data:b}}function cL(){try{return new a.XMLHttpRequest}catch(b){}}function cM(){try{return new a.ActiveXObject("Microsoft.XMLHTTP")}catch(b){}}function cU(){return setTimeout(function(){cN=b},0),cN=p.now()}function cV(a,b){p.each(b,function(b,c){var d=(cT[b]||[]).concat(cT["*"]),e=0,f=d.length;for(;e<f;e++)if(d[e].call(a,b,c))return})}function cW(a,b,c){var d,e=0,f=0,g=cS.length,h=p.Deferred().always(function(){delete i.elem}),i=function(){var b=cN||cU(),c=Math.max(0,j.startTime+j.duration-b),d=1-(c/j.duration||0),e=0,f=j.tweens.length;for(;e<f;e++)j.tweens[e].run(d);return h.notifyWith(a,[j,d,c]),d<1&&f?c:(h.resolveWith(a,[j]),!1)},j=h.promise({elem:a,props:p.extend({},b),opts:p.extend(!0,{specialEasing:{}},c),originalProperties:b,originalOptions:c,startTime:cN||cU(),duration:c.duration,tweens:[],createTween:function(b,c,d){var e=p.Tween(a,j.opts,b,c,j.opts.specialEasing[b]||j.opts.easing);return j.tweens.push(e),e},stop:function(b){var c=0,d=b?j.tweens.length:0;for(;c<
 d;c++)j.tweens[c].run(1);return b?h.resolveWith(a,[j,b]):h.rejectWith(a,[j,b]),this}}),k=j.props;cX(k,j.opts.specialEasing);for(;e<g;e++){d=cS[e].call(j,a,k,j.opts);if(d)return d}return cV(j,k),p.isFunction(j.opts.start)&&j.opts.start.call(a,j),p.fx.timer(p.extend(i,{anim:j,queue:j.opts.queue,elem:a})),j.progress(j.opts.progress).done(j.opts.done,j.opts.complete).fail(j.opts.fail).always(j.opts.always)}function cX(a,b){var c,d,e,f,g;for(c in a){d=p.camelCase(c),e=b[d],f=a[c],p.isArray(f)&&(e=f[1],f=a[c]=f[0]),c!==d&&(a[d]=f,delete a[c]),g=p.cssHooks[d];if(g&&"expand"in g){f=g.expand(f),delete a[d];for(c in f)c in a||(a[c]=f[c],b[c]=e)}else b[d]=e}}function cY(a,b,c){var d,e,f,g,h,i,j,k,l=this,m=a.style,n={},o=[],q=a.nodeType&&bZ(a);c.queue||(j=p._queueHooks(a,"fx"),j.unqueued==null&&(j.unqueued=0,k=j.empty.fire,j.empty.fire=function(){j.unqueued||k()}),j.unqueued++,l.always(function(){l.always(function(){j.unqueued--,p.queue(a,"fx").length||j.empty.fire()})})),a.nodeType===1&&("heig
 ht"in b||"width"in b)&&(c.overflow=[m.overflow,m.overflowX,m.overflowY],p.css(a,"display")==="inline"&&p.css(a,"float")==="none"&&(!p.support.inlineBlockNeedsLayout||cc(a.nodeName)==="inline"?m.display="inline-block":m.zoom=1)),c.overflow&&(m.overflow="hidden",p.support.shrinkWrapBlocks||l.done(function(){m.overflow=c.overflow[0],m.overflowX=c.overflow[1],m.overflowY=c.overflow[2]}));for(d in b){f=b[d];if(cP.exec(f)){delete b[d];if(f===(q?"hide":"show"))continue;o.push(d)}}g=o.length;if(g){h=p._data(a,"fxshow")||p._data(a,"fxshow",{}),q?p(a).show():l.done(function(){p(a).hide()}),l.done(function(){var b;p.removeData(a,"fxshow",!0);for(b in n)p.style(a,b,n[b])});for(d=0;d<g;d++)e=o[d],i=l.createTween(e,q?h[e]:0),n[e]=h[e]||p.style(a,e),e in h||(h[e]=i.start,q&&(i.end=i.start,i.start=e==="width"||e==="height"?1:0))}}function cZ(a,b,c,d,e){return new cZ.prototype.init(a,b,c,d,e)}function c$(a,b){var c,d={height:a},e=0;b=b?1:0;for(;e<4;e+=2-b)c=bV[e],d["margin"+c]=d["padding"+c]=a;retur
 n b&&(d.opacity=d.width=a),d}function da(a){return p.isWindow(a)?a:a.nodeType===9?a.defaultView||a.parentWindow:!1}var c,d,e=a.document,f=a.location,g=a.navigator,h=a.jQuery,i=a.$,j=Array.prototype.push,k=Array.prototype.slice,l=Array.prototype.indexOf,m=Object.prototype.toString,n=Object.prototype.hasOwnProperty,o=String.prototype.trim,p=function(a,b){return new p.fn.init(a,b,c)},q=/[\-+]?(?:\d*\.|)\d+(?:[eE][\-+]?\d+|)/.source,r=/\S/,s=/\s+/,t=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,u=/^(?:[^#<]*(<[\w\W]+>)[^>]*$|#([\w\-]*)$)/,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^[\],:{}\s]*$/,x=/(?:^|:|,)(?:\s*\[)+/g,y=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,z=/"[^"\\\r\n]*"|true|false|null|-?(?:\d\d*\.|)\d+(?:[eE][\-+]?\d+|)/g,A=/^-ms-/,B=/-([\da-z])/gi,C=function(a,b){return(b+"").toUpperCase()},D=function(){e.addEventListener?(e.removeEventListener("DOMContentLoaded",D,!1),p.ready()):e.readyState==="complete"&&(e.detachEvent("onreadystatechange",D),p.ready())},E={};p.fn=p.prototype={constructor:p,init
 :function(a,c,d){var f,g,h,i;if(!a)return this;if(a.nodeType)return this.context=this[0]=a,this.length=1,this;if(typeof a=="string"){a.charAt(0)==="<"&&a.charAt(a.length-1)===">"&&a.length>=3?f=[null,a,null]:f=u.exec(a);if(f&&(f[1]||!c)){if(f[1])return c=c instanceof p?c[0]:c,i=c&&c.nodeType?c.ownerDocument||c:e,a=p.parseHTML(f[1],i,!0),v.test(f[1])&&p.isPlainObject(c)&&this.attr.call(a,c,!0),p.merge(this,a);g=e.getElementById(f[2]);if(g&&g.parentNode){if(g.id!==f[2])return d.find(a);this.length=1,this[0]=g}return this.context=e,this.selector=a,this}return!c||c.jquery?(c||d).find(a):this.constructor(c).find(a)}return p.isFunction(a)?d.ready(a):(a.selector!==b&&(this.selector=a.selector,this.context=a.context),p.makeArray(a,this))},selector:"",jquery:"1.8.2",length:0,size:function(){return this.length},toArray:function(){return k.call(this)},get:function(a){return a==null?this.toArray():a<0?this[this.length+a]:this[a]},pushStack:function(a,b,c){var d=p.merge(this.constructor(),a);ret
 urn d.prevObject=this,d.context=this.context,b==="find"?d.selector=this.selector+(this.selector?" ":"")+c:b&&(d.selector=this.selector+"."+b+"("+c+")"),d},each:function(a,b){return p.each(this,a,b)},ready:function(a){return p.ready.promise().done(a),this},eq:function(a){return a=+a,a===-1?this.slice(a):this.slice(a,a+1)},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},slice:function(){return this.pushStack(k.apply(this,arguments),"slice",k.call(arguments).join(","))},map:function(a){return this.pushStack(p.map(this,function(b,c){return a.call(b,c,b)}))},end:function(){return this.prevObject||this.constructor(null)},push:j,sort:[].sort,splice:[].splice},p.fn.init.prototype=p.fn,p.extend=p.fn.extend=function(){var a,c,d,e,f,g,h=arguments[0]||{},i=1,j=arguments.length,k=!1;typeof h=="boolean"&&(k=h,h=arguments[1]||{},i=2),typeof h!="object"&&!p.isFunction(h)&&(h={}),j===i&&(h=this,--i);for(;i<j;i++)if((a=arguments[i])!=null)for(c in a){d=h[c],e=a[c];if(h===e)con
 tinue;k&&e&&(p.isPlainObject(e)||(f=p.isArray(e)))?(f?(f=!1,g=d&&p.isArray(d)?d:[]):g=d&&p.isPlainObject(d)?d:{},h[c]=p.extend(k,g,e)):e!==b&&(h[c]=e)}return h},p.extend({noConflict:function(b){return a.$===p&&(a.$=i),b&&a.jQuery===p&&(a.jQuery=h),p},isReady:!1,readyWait:1,holdReady:function(a){a?p.readyWait++:p.ready(!0)},ready:function(a){if(a===!0?--p.readyWait:p.isReady)return;if(!e.body)return setTimeout(p.ready,1);p.isReady=!0;if(a!==!0&&--p.readyWait>0)return;d.resolveWith(e,[p]),p.fn.trigger&&p(e).trigger("ready").off("ready")},isFunction:function(a){return p.type(a)==="function"},isArray:Array.isArray||function(a){return p.type(a)==="array"},isWindow:function(a){return a!=null&&a==a.window},isNumeric:function(a){return!isNaN(parseFloat(a))&&isFinite(a)},type:function(a){return a==null?String(a):E[m.call(a)]||"object"},isPlainObject:function(a){if(!a||p.type(a)!=="object"||a.nodeType||p.isWindow(a))return!1;try{if(a.constructor&&!n.call(a,"constructor")&&!n.call(a.constructo
 r.prototype,"isPrototypeOf"))return!1}catch(c){return!1}var d;for(d in a);return d===b||n.call(a,d)},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},error:function(a){throw new Error(a)},parseHTML:function(a,b,c){var d;return!a||typeof a!="string"?null:(typeof b=="boolean"&&(c=b,b=0),b=b||e,(d=v.exec(a))?[b.createElement(d[1])]:(d=p.buildFragment([a],b,c?null:[]),p.merge([],(d.cacheable?p.clone(d.fragment):d.fragment).childNodes)))},parseJSON:function(b){if(!b||typeof b!="string")return null;b=p.trim(b);if(a.JSON&&a.JSON.parse)return a.JSON.parse(b);if(w.test(b.replace(y,"@").replace(z,"]").replace(x,"")))return(new Function("return "+b))();p.error("Invalid JSON: "+b)},parseXML:function(c){var d,e;if(!c||typeof c!="string")return null;try{a.DOMParser?(e=new DOMParser,d=e.parseFromString(c,"text/xml")):(d=new ActiveXObject("Microsoft.XMLDOM"),d.async="false",d.loadXML(c))}catch(f){d=b}return(!d||!d.documentElement||d.getElementsByTagName("parsererror").length)&&p.error(
 "Invalid XML: "+c),d},noop:function(){},globalEval:function(b){b&&r.test(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(A,"ms-").replace(B,C)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,c,d){var e,f=0,g=a.length,h=g===b||p.isFunction(a);if(d){if(h){for(e in a)if(c.apply(a[e],d)===!1)break}else for(;f<g;)if(c.apply(a[f++],d)===!1)break}else if(h){for(e in a)if(c.call(a[e],e,a[e])===!1)break}else for(;f<g;)if(c.call(a[f],f,a[f++])===!1)break;return a},trim:o&&!o.call("\ufeff�")?function(a){return a==null?"":o.call(a)}:function(a){return a==null?"":(a+"").replace(t,"")},makeArray:function(a,b){var c,d=b||[];return a!=null&&(c=p.type(a),a.length==null||c==="string"||c==="function"||c==="regexp"||p.isWindow(a)?j.call(d,a):p.merge(d,a)),d},inArray:function(a,b,c){var d;if(b){if(l)return l.call(b,a,c);d=b.length,c=c?c<0?Math.max(0,d+c):c:0;for(;c<d;c++)if(c in b&&b[c]===a)return c}return-
 1},merge:function(a,c){var d=c.length,e=a.length,f=0;if(typeof d=="number")for(;f<d;f++)a[e++]=c[f];else while(c[f]!==b)a[e++]=c[f++];return a.length=e,a},grep:function(a,b,c){var d,e=[],f=0,g=a.length;c=!!c;for(;f<g;f++)d=!!b(a[f],f),c!==d&&e.push(a[f]);return e},map:function(a,c,d){var e,f,g=[],h=0,i=a.length,j=a instanceof p||i!==b&&typeof i=="number"&&(i>0&&a[0]&&a[i-1]||i===0||p.isArray(a));if(j)for(;h<i;h++)e=c(a[h],h,d),e!=null&&(g[g.length]=e);else for(f in a)e=c(a[f],f,d),e!=null&&(g[g.length]=e);return g.concat.apply([],g)},guid:1,proxy:function(a,c){var d,e,f;return typeof c=="string"&&(d=a[c],c=a,a=d),p.isFunction(a)?(e=k.call(arguments,2),f=function(){return a.apply(c,e.concat(k.call(arguments)))},f.guid=a.guid=a.guid||p.guid++,f):b},access:function(a,c,d,e,f,g,h){var i,j=d==null,k=0,l=a.length;if(d&&typeof d=="object"){for(k in d)p.access(a,c,k,d[k],1,g,e);f=1}else if(e!==b){i=h===b&&p.isFunction(e),j&&(i?(i=c,c=function(a,b,c){return i.call(p(a),c)}):(c.call(a,e),c=nu
 ll));if(c)for(;k<l;k++)c(a[k],d,i?e.call(a[k],k,c(a[k],d)):e,h);f=1}return f?a:j?c.call(a):l?c(a[0],d):g},now:function(){return(new Date).getTime()}}),p.ready.promise=function(b){if(!d){d=p.Deferred();if(e.readyState==="complete")setTimeout(p.ready,1);else if(e.addEventListener)e.addEventListener("DOMContentLoaded",D,!1),a.addEventListener("load",p.ready,!1);else{e.attachEvent("onreadystatechange",D),a.attachEvent("onload",p.ready);var c=!1;try{c=a.frameElement==null&&e.documentElement}catch(f){}c&&c.doScroll&&function g(){if(!p.isReady){try{c.doScroll("left")}catch(a){return setTimeout(g,50)}p.ready()}}()}}return d.promise(b)},p.each("Boolean Number String Function Array Date RegExp Object".split(" "),function(a,b){E["[object "+b+"]"]=b.toLowerCase()}),c=p(e);var F={};p.Callbacks=function(a){a=typeof a=="string"?F[a]||G(a):p.extend({},a);var c,d,e,f,g,h,i=[],j=!a.once&&[],k=function(b){c=a.memory&&b,d=!0,h=f||0,f=0,g=i.length,e=!0;for(;i&&h<g;h++)if(i[h].apply(b[0],b[1])===!1&&a.st
 opOnFalse){c=!1;break}e=!1,i&&(j?j.length&&k(j.shift()):c?i=[]:l.disable())},l={add:function(){if(i){var b=i.length;(function d(b){p.each(b,function(b,c){var e=p.type(c);e==="function"&&(!a.unique||!l.has(c))?i.push(c):c&&c.length&&e!=="string"&&d(c)})})(arguments),e?g=i.length:c&&(f=b,k(c))}return this},remove:function(){return i&&p.each(arguments,function(a,b){var c;while((c=p.inArray(b,i,c))>-1)i.splice(c,1),e&&(c<=g&&g--,c<=h&&h--)}),this},has:function(a){return p.inArray(a,i)>-1},empty:function(){return i=[],this},disable:function(){return i=j=c=b,this},disabled:function(){return!i},lock:function(){return j=b,c||l.disable(),this},locked:function(){return!j},fireWith:function(a,b){return b=b||[],b=[a,b.slice?b.slice():b],i&&(!d||j)&&(e?j.push(b):k(b)),this},fire:function(){return l.fireWith(this,arguments),this},fired:function(){return!!d}};return l},p.extend({Deferred:function(a){var b=[["resolve","done",p.Callbacks("once memory"),"resolved"],["reject","fail",p.Callbacks("once 
 memory"),"rejected"],["notify","progress",p.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return p.Deferred(function(c){p.each(b,function(b,d){var f=d[0],g=a[b];e[d[1]](p.isFunction(g)?function(){var a=g.apply(this,arguments);a&&p.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f+"With"](this===e?c:this,[a])}:c[f])}),a=null}).promise()},promise:function(a){return a!=null?p.extend(a,d):d}},e={};return d.pipe=d.then,p.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[a^1][2].disable,b[2][2].lock),e[f[0]]=g.fire,e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=k.call(arguments),d=c.length,e=d!==1||a&&p.isFunction(a.promise)?d:0,f=e===1?a:p.Deferred(),g=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?k.call(arguments):d,c===h?f.notifyWith(b,c):--e||f
 .resolveWith(b,c)}},h,i,j;if(d>1){h=new Array(d),i=new Array(d),j=new Array(d);for(;b<d;b++)c[b]&&p.isFunction(c[b].promise)?c[b].promise().done(g(b,j,c)).fail(f.reject).progress(g(b,i,h)):--e}return e||f.resolveWith(j,c),f.promise()}}),p.support=function(){var b,c,d,f,g,h,i,j,k,l,m,n=e.createElement("div");n.setAttribute("className","t"),n.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",c=n.getElementsByTagName("*"),d=n.getElementsByTagName("a")[0],d.style.cssText="top:1px;float:left;opacity:.5";if(!c||!c.length)return{};f=e.createElement("select"),g=f.appendChild(e.createElement("option")),h=n.getElementsByTagName("input")[0],b={leadingWhitespace:n.firstChild.nodeType===3,tbody:!n.getElementsByTagName("tbody").length,htmlSerialize:!!n.getElementsByTagName("link").length,style:/top/.test(d.getAttribute("style")),hrefNormalized:d.getAttribute("href")==="/a",opacity:/^0.5/.test(d.style.opacity),cssFloat:!!d.style.cssFloat,checkOn:h.value==="on",optSelec
 ted:g.selected,getSetAttribute:n.className!=="t",enctype:!!e.createElement("form").enctype,html5Clone:e.createElement("nav").cloneNode(!0).outerHTML!=="<:nav></:nav>",boxModel:e.compatMode==="CSS1Compat",submitBubbles:!0,changeBubbles:!0,focusinBubbles:!1,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},h.checked=!0,b.noCloneChecked=h.cloneNode(!0).checked,f.disabled=!0,b.optDisabled=!g.disabled;try{delete n.test}catch(o){b.deleteExpando=!1}!n.addEventListener&&n.attachEvent&&n.fireEvent&&(n.attachEvent("onclick",m=function(){b.noCloneEvent=!1}),n.cloneNode(!0).fireEvent("onclick"),n.detachEvent("onclick",m)),h=e.createElement("input"),h.value="t",h.setAttribute("type","radio"),b.radioValue=h.value==="t",h.setAttribute("checked","checked"),h.setAttribute("name","t"),n.appendChild(h),i=e.createDocumentFragment(),i.appendChild(n.lastChild),b.checkClone=i.cloneNode(!0).cloneNode(!0).lastChild.ch
 ecked,b.appendChecked=h.checked,i.removeChild(h),i.appendChild(n);if(n.attachEvent)for(k in{submit:!0,change:!0,focusin:!0})j="on"+k,l=j in n,l||(n.setAttribute(j,"return;"),l=typeof n[j]=="function"),b[k+"Bubbles"]=l;return p(function(){var c,d,f,g,h="padding:0;margin:0;border:0;display:block;overflow:hidden;",i=e.getElementsByTagName("body")[0];if(!i)return;c=e.createElement("div"),c.style.cssText="visibility:hidden;border:0;width:0;height:0;position:static;top:0;margin-top:1px",i.insertBefore(c,i.firstChild),d=e.createElement("div"),c.appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",f=d.getElementsByTagName("td"),f[0].style.cssText="padding:0;margin:0;border:0;display:none",l=f[0].offsetHeight===0,f[0].style.display="",f[1].style.display="none",b.reliableHiddenOffsets=l&&f[0].offsetHeight===0,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-t
 op:1%;position:absolute;top:1%;",b.boxSizing=d.offsetWidth===4,b.doesNotIncludeMarginInBodyOffset=i.offsetTop!==1,a.getComputedStyle&&(b.pixelPosition=(a.getComputedStyle(d,null)||{}).top!=="1%",b.boxSizingReliable=(a.getComputedStyle(d,null)||{width:"4px"}).width==="4px",g=e.createElement("div"),g.style.cssText=d.style.cssText=h,g.style.marginRight=g.style.width="0",d.style.width="1px",d.appendChild(g),b.reliableMarginRight=!parseFloat((a.getComputedStyle(g,null)||{}).marginRight)),typeof d.style.zoom!="undefined"&&(d.innerHTML="",d.style.cssText=h+"width:1px;padding:1px;display:inline;zoom:1",b.inlineBlockNeedsLayout=d.offsetWidth===3,d.style.display="block",d.style.overflow="visible",d.innerHTML="<div></div>",d.firstChild.style.width="5px",b.shrinkWrapBlocks=d.offsetWidth!==3,c.style.zoom=1),i.removeChild(c),c=d=f=g=null}),i.removeChild(n),c=d=f=g=h=i=n=null,b}();var H=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,I=/([A-Z])/g;p.extend({cache:{},deletedIds:[],uuid:0,expando:"jQuery"+(p.fn.jquer
 y+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(a){return a=a.nodeType?p.cache[a[p.expando]]:a[p.expando],!!a&&!K(a)},data:function(a,c,d,e){if(!p.acceptData(a))return;var f,g,h=p.expando,i=typeof c=="string",j=a.nodeType,k=j?p.cache:a,l=j?a[h]:a[h]&&h;if((!l||!k[l]||!e&&!k[l].data)&&i&&d===b)return;l||(j?a[h]=l=p.deletedIds.pop()||p.guid++:l=h),k[l]||(k[l]={},j||(k[l].toJSON=p.noop));if(typeof c=="object"||typeof c=="function")e?k[l]=p.extend(k[l],c):k[l].data=p.extend(k[l].data,c);return f=k[l],e||(f.data||(f.data={}),f=f.data),d!==b&&(f[p.camelCase(c)]=d),i?(g=f[c],g==null&&(g=f[p.camelCase(c)])):g=f,g},removeData:function(a,b,c){if(!p.acceptData(a))return;var d,e,f,g=a.nodeType,h=g?p.cache:a,i=g?a[p.expando]:p.expando;if(!h[i])return;if(b){d=c?h[i]:h[i].data;if(d){p.isArray(b)||(b in d?b=[b]:(b=p.camelCase(b),b in d?b=[b]:b=b.split(" ")));for(e=0,f=b.length;e<f;e++)delete d[b[e]];if(!(c?K:p.isEmp
 tyObject)(d))return}}if(!c){delete h[i].data;if(!K(h[i]))return}g?p.cleanData([a],!0):p.support.deleteExpando||h!=h.window?delete h[i]:h[i]=null},_data:function(a,b,c){return p.data(a,b,c,!0)},acceptData:function(a){var b=a.nodeName&&p.noData[a.nodeName.toLowerCase()];return!b||b!==!0&&a.getAttribute("classid")===b}}),p.fn.extend({data:function(a,c){var d,e,f,g,h,i=this[0],j=0,k=null;if(a===b){if(this.length){k=p.data(i);if(i.nodeType===1&&!p._data(i,"parsedAttrs")){f=i.attributes;for(h=f.length;j<h;j++)g=f[j].name,g.indexOf("data-")||(g=p.camelCase(g.substring(5)),J(i,g,k[g]));p._data(i,"parsedAttrs",!0)}}return k}return typeof a=="object"?this.each(function(){p.data(this,a)}):(d=a.split(".",2),d[1]=d[1]?"."+d[1]:"",e=d[1]+"!",p.access(this,function(c){if(c===b)return k=this.triggerHandler("getData"+e,[d[0]]),k===b&&i&&(k=p.data(i,a),k=J(i,a,k)),k===b&&d[1]?this.data(d[0]):k;d[1]=c,this.each(function(){var b=p(this);b.triggerHandler("setData"+e,d),p.data(this,a,c),b.triggerHandler(
 "changeData"+e,d)})},null,c,arguments.length>1,null,!1))},removeData:function(a){return this.each(function(){p.removeData(this,a)})}}),p.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=p._data(a,b),c&&(!d||p.isArray(c)?d=p._data(a,b,p.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||"fx";var c=p.queue(a,b),d=c.length,e=c.shift(),f=p._queueHooks(a,b),g=function(){p.dequeue(a,b)};e==="inprogress"&&(e=c.shift(),d--),e&&(b==="fx"&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return p._data(a,c)||p._data(a,c,{empty:p.Callbacks("once memory").add(function(){p.removeData(a,b+"queue",!0),p.removeData(a,c,!0)})})}}),p.fn.extend({queue:function(a,c){var d=2;return typeof a!="string"&&(c=a,a="fx",d--),arguments.length<d?p.queue(this[0],a):c===b?this:this.each(function(){var b=p.queue(this,a,c);p._queueHooks(this,a),a==="fx"&&b[0]!=="inprogress"&&p.dequeue(this,a)})},dequeue:function(a)
 {return this.each(function(){p.dequeue(this,a)})},delay:function(a,b){return a=p.fx?p.fx.speeds[a]||a:a,b=b||"fx",this.queue(b,function(b,c){var d=setTimeout(b,a);c.stop=function(){clearTimeout(d)}})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,c){var d,e=1,f=p.Deferred(),g=this,h=this.length,i=function(){--e||f.resolveWith(g,[g])};typeof a!="string"&&(c=a,a=b),a=a||"fx";while(h--)d=p._data(g[h],a+"queueHooks"),d&&d.empty&&(e++,d.empty.add(i));return i(),f.promise(c)}});var L,M,N,O=/[\t\r\n]/g,P=/\r/g,Q=/^(?:button|input)$/i,R=/^(?:button|input|object|select|textarea)$/i,S=/^a(?:rea|)$/i,T=/^(?:autofocus|autoplay|async|checked|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped|selected)$/i,U=p.support.getSetAttribute;p.fn.extend({attr:function(a,b){return p.access(this,p.attr,a,b,arguments.length>1)},removeAttr:function(a){return this.each(function(){p.removeAttr(this,a)})},prop:function(a,b){return p.access(this,p.prop,a,b,argume
 nts.length>1)},removeProp:function(a){return a=p.propFix[a]||a,this.each(function(){try{this[a]=b,delete this[a]}catch(c){}})},addClass:function(a){var b,c,d,e,f,g,h;if(p.isFunction(a))return this.each(function(b){p(this).addClass(a.call(this,b,this.className))});if(a&&typeof a=="string"){b=a.split(s);for(c=0,d=this.length;c<d;c++){e=this[c];if(e.nodeType===1)if(!e.className&&b.length===1)e.className=a;else{f=" "+e.className+" ";for(g=0,h=b.length;g<h;g++)f.indexOf(" "+b[g]+" ")<0&&(f+=b[g]+" ");e.className=p.trim(f)}}}return this},removeClass:function(a){var c,d,e,f,g,h,i;if(p.isFunction(a))return this.each(function(b){p(this).removeClass(a.call(this,b,this.className))});if(a&&typeof a=="string"||a===b){c=(a||"").split(s);for(h=0,i=this.length;h<i;h++){e=this[h];if(e.nodeType===1&&e.className){d=(" "+e.className+" ").replace(O," ");for(f=0,g=c.length;f<g;f++)while(d.indexOf(" "+c[f]+" ")>=0)d=d.replace(" "+c[f]+" "," ");e.className=a?p.trim(d):""}}}return this},toggleClass:function
 (a,b){var c=typeof a,d=typeof b=="boolean";return p.isFunction(a)?this.each(function(c){p(this).toggleClass(a.call(this,c,this.className,b),b)}):this.each(function(){if(c==="string"){var e,f=0,g=p(this),h=b,i=a.split(s);while(e=i[f++])h=d?h:!g.hasClass(e),g[h?"addClass":"removeClass"](e)}else if(c==="undefined"||c==="boolean")this.className&&p._data(this,"__className__",this.className),this.className=this.className||a===!1?"":p._data(this,"__className__")||""})},hasClass:function(a){var b=" "+a+" ",c=0,d=this.length;for(;c<d;c++)if(this[c].nodeType===1&&(" "+this[c].className+" ").replace(O," ").indexOf(b)>=0)return!0;return!1},val:function(a){var c,d,e,f=this[0];if(!arguments.length){if(f)return c=p.valHooks[f.type]||p.valHooks[f.nodeName.toLowerCase()],c&&"get"in c&&(d=c.get(f,"value"))!==b?d:(d=f.value,typeof d=="string"?d.replace(P,""):d==null?"":d);return}return e=p.isFunction(a),this.each(function(d){var f,g=p(this);if(this.nodeType!==1)return;e?f=a.call(this,d,g.val()):f=a,f=
 =null?f="":typeof f=="number"?f+="":p.isArray(f)&&(f=p.map(f,function(a){return a==null?"":a+""})),c=p.valHooks[this.type]||p.valHooks[this.nodeName.toLowerCase()];if(!c||!("set"in c)||c.set(this,f,"value")===b)this.value=f})}}),p.extend({valHooks:{option:{get:function(a){var b=a.attributes.value;return!b||b.specified?a.value:a.text}},select:{get:function(a){var b,c,d,e,f=a.selectedIndex,g=[],h=a.options,i=a.type==="select-one";if(f<0)return null;c=i?f:0,d=i?f+1:h.length;for(;c<d;c++){e=h[c];if(e.selected&&(p.support.optDisabled?!e.disabled:e.getAttribute("disabled")===null)&&(!e.parentNode.disabled||!p.nodeName(e.parentNode,"optgroup"))){b=p(e).val();if(i)return b;g.push(b)}}return i&&!g.length&&h.length?p(h[f]).val():g},set:function(a,b){var c=p.makeArray(b);return p(a).find("option").each(function(){this.selected=p.inArray(p(this).val(),c)>=0}),c.length||(a.selectedIndex=-1),c}}},attrFn:{},attr:function(a,c,d,e){var f,g,h,i=a.nodeType;if(!a||i===3||i===8||i===2)return;if(e&&p.isF
 unction(p.fn[c]))return p(a)[c](d);if(typeof a.getAttribute=="undefined")return p.prop(a,c,d);h=i!==1||!p.isXMLDoc(a),h&&(c=c.toLowerCase(),g=p.attrHooks[c]||(T.test(c)?M:L));if(d!==b){if(d===null){p.removeAttr(a,c);return}return g&&"set"in g&&h&&(f=g.set(a,d,c))!==b?f:(a.setAttribute(c,d+""),d)}return g&&"get"in g&&h&&(f=g.get(a,c))!==null?f:(f=a.getAttribute(c),f===null?b:f)},removeAttr:function(a,b){var c,d,e,f,g=0;if(b&&a.nodeType===1){d=b.split(s);for(;g<d.length;g++)e=d[g],e&&(c=p.propFix[e]||e,f=T.test(e),f||p.attr(a,e,""),a.removeAttribute(U?e:c),f&&c in a&&(a[c]=!1))}},attrHooks:{type:{set:function(a,b){if(Q.test(a.nodeName)&&a.parentNode)p.error("type property can't be changed");else if(!p.support.radioValue&&b==="radio"&&p.nodeName(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}},value:{get:function(a,b){return L&&p.nodeName(a,"button")?L.get(a,b):b in a?a.value:null},set:function(a,b,c){if(L&&p.nodeName(a,"button"))return L.set(a,b,c);a.value
 =b}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(a,c,d){var e,f,g,h=a.nodeType;if(!a||h===3||h===8||h===2)return;return g=h!==1||!p.isXMLDoc(a),g&&(c=p.propFix[c]||c,f=p.propHooks[c]),d!==b?f&&"set"in f&&(e=f.set(a,d,c))!==b?e:a[c]=d:f&&"get"in f&&(e=f.get(a,c))!==null?e:a[c]},propHooks:{tabIndex:{get:function(a){var c=a.getAttributeNode("tabindex");return c&&c.specified?parseInt(c.value,10):R.test(a.nodeName)||S.test(a.nodeName)&&a.href?0:b}}}}),M={get:function(a,c){var d,e=p.prop(a,c);return e===!0||typeof e!="boolean"&&(d=a.getAttributeNode(c))&&d.nodeValue!==!1?c.toLowerCase():b},set:function(a,b,c){var d;return b===!1?p.removeAttr(a,c):(d=p.propFix[c]||c,d in a&&(a[d]=!0),a.setAttribute(c,c.toLowerCase())),c}},U||(N={name:!0,id:!0,coords:!
 0},L=p.valHooks.button={get:function(a,c){var d;return d=a.getAttributeNode(c),d&&(N[c]?d.value!=="":d.specified)?d.value:b},set:function(a,b,c){var d=a.getAttributeNode(c);return d||(d=e.createAttribute(c),a.setAttributeNode(d)),d.value=b+""}},p.each(["width","height"],function(a,b){p.attrHooks[b]=p.extend(p.attrHooks[b],{set:function(a,c){if(c==="")return a.setAttribute(b,"auto"),c}})}),p.attrHooks.contenteditable={get:L.get,set:function(a,b,c){b===""&&(b="false"),L.set(a,b,c)}}),p.support.hrefNormalized||p.each(["href","src","width","height"],function(a,c){p.attrHooks[c]=p.extend(p.attrHooks[c],{get:function(a){var d=a.getAttribute(c,2);return d===null?b:d}})}),p.support.style||(p.attrHooks.style={get:function(a){return a.style.cssText.toLowerCase()||b},set:function(a,b){return a.style.cssText=b+""}}),p.support.optSelected||(p.propHooks.selected=p.extend(p.propHooks.selected,{get:function(a){var b=a.parentNode;return b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex),n
 ull}})),p.support.enctype||(p.propFix.enctype="encoding"),p.support.checkOn||p.each(["radio","checkbox"],function(){p.valHooks[this]={get:function(a){return a.getAttribute("value")===null?"on":a.value}}}),p.each(["radio","checkbox"],function(){p.valHooks[this]=p.extend(p.valHooks[this],{set:function(a,b){if(p.isArray(b))return a.checked=p.inArray(p(a).val(),b)>=0}})});var V=/^(?:textarea|input|select)$/i,W=/^([^\.]*|)(?:\.(.+)|)$/,X=/(?:^|\s)hover(\.\S+|)\b/,Y=/^key/,Z=/^(?:mouse|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=function(a){return p.event.special.hover?a:a.replace(X,"mouseenter$1 mouseleave$1")};p.event={add:function(a,c,d,e,f){var g,h,i,j,k,l,m,n,o,q,r;if(a.nodeType===3||a.nodeType===8||!c||!d||!(g=p._data(a)))return;d.handler&&(o=d,d=o.handler,f=o.selector),d.guid||(d.guid=p.guid++),i=g.events,i||(g.events=i={}),h=g.handle,h||(g.handle=h=function(a){return typeof p!="undefined"&&(!a||p.event.triggered!==a.type)?p.event.dispatch.apply(h.elem,arguments):b},h
 .elem=a),c=p.trim(_(c)).split(" ");for(j=0;j<c.length;j++){k=W.exec(c[j])||[],l=k[1],m=(k[2]||"").split(".").sort(),r=p.event.special[l]||{},l=(f?r.delegateType:r.bindType)||l,r=p.event.special[l]||{},n=p.extend({type:l,origType:k[1],data:e,handler:d,guid:d.guid,selector:f,needsContext:f&&p.expr.match.needsContext.test(f),namespace:m.join(".")},o),q=i[l];if(!q){q=i[l]=[],q.delegateCount=0;if(!r.setup||r.setup.call(a,e,m,h)===!1)a.addEventListener?a.addEventListener(l,h,!1):a.attachEvent&&a.attachEvent("on"+l,h)}r.add&&(r.add.call(a,n),n.handler.guid||(n.handler.guid=d.guid)),f?q.splice(q.delegateCount++,0,n):q.push(n),p.event.global[l]=!0}a=null},global:{},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,q,r=p.hasData(a)&&p._data(a);if(!r||!(m=r.events))return;b=p.trim(_(b||"")).split(" ");for(f=0;f<b.length;f++){g=W.exec(b[f])||[],h=i=g[1],j=g[2];if(!h){for(h in m)p.event.remove(a,h+b[f],c,d,!0);continue}n=p.event.special[h]||{},h=(d?n.delegateType:n.bindType)||h,o=m[h]||[],k=o.l
 ength,j=j?new RegExp("(^|\\.)"+j.split(".").sort().join("\\.(?:.*\\.|)")+"(\\.|$)"):null;for(l=0;l<o.length;l++)q=o[l],(e||i===q.origType)&&(!c||c.guid===q.guid)&&(!j||j.test(q.namespace))&&(!d||d===q.selector||d==="**"&&q.selector)&&(o.splice(l--,1),q.selector&&o.delegateCount--,n.remove&&n.remove.call(a,q));o.length===0&&k!==o.length&&((!n.teardown||n.teardown.call(a,j,r.handle)===!1)&&p.removeEvent(a,h,r.handle),delete m[h])}p.isEmptyObject(m)&&(delete r.handle,p.removeData(a,"events",!0))},customEvent:{getData:!0,setData:!0,changeData:!0},trigger:function(c,d,f,g){if(!f||f.nodeType!==3&&f.nodeType!==8){var h,i,j,k,l,m,n,o,q,r,s=c.type||c,t=[];if($.test(s+p.event.triggered))return;s.indexOf("!")>=0&&(s=s.slice(0,-1),i=!0),s.indexOf(".")>=0&&(t=s.split("."),s=t.shift(),t.sort());if((!f||p.event.customEvent[s])&&!p.event.global[s])return;c=typeof c=="object"?c[p.expando]?c:new p.Event(s,c):new p.Event(s),c.type=s,c.isTrigger=!0,c.exclusive=i,c.namespace=t.join("."),c.namespace_re=c
 .namespace?new RegExp("(^|\\.)"+t.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,m=s.indexOf(":")<0?"on"+s:"";if(!f){h=p.cache;for(j in h)h[j].events&&h[j].events[s]&&p.event.trigger(c,d,h[j].handle.elem,!0);return}c.result=b,c.target||(c.target=f),d=d!=null?p.makeArray(d):[],d.unshift(c),n=p.event.special[s]||{};if(n.trigger&&n.trigger.apply(f,d)===!1)return;q=[[f,n.bindType||s]];if(!g&&!n.noBubble&&!p.isWindow(f)){r=n.delegateType||s,k=$.test(r+s)?f:f.parentNode;for(l=f;k;k=k.parentNode)q.push([k,r]),l=k;l===(f.ownerDocument||e)&&q.push([l.defaultView||l.parentWindow||a,r])}for(j=0;j<q.length&&!c.isPropagationStopped();j++)k=q[j][0],c.type=q[j][1],o=(p._data(k,"events")||{})[c.type]&&p._data(k,"handle"),o&&o.apply(k,d),o=m&&k[m],o&&p.acceptData(k)&&o.apply&&o.apply(k,d)===!1&&c.preventDefault();return c.type=s,!g&&!c.isDefaultPrevented()&&(!n._default||n._default.apply(f.ownerDocument,d)===!1)&&(s!=="click"||!p.nodeName(f,"a"))&&p.acceptData(f)&&m&&f[s]&&(s!=="focus"&&s!=="blur"||c.target.
 offsetWidth!==0)&&!p.isWindow(f)&&(l=f[m],l&&(f[m]=null),p.event.triggered=s,f[s](),p.event.triggered=b,l&&(f[m]=l)),c.result}return},dispatch:function(c){c=p.event.fix(c||a.event);var d,e,f,g,h,i,j,l,m,n,o=(p._data(this,"events")||{})[c.type]||[],q=o.delegateCount,r=k.call(arguments),s=!c.exclusive&&!c.namespace,t=p.event.special[c.type]||{},u=[];r[0]=c,c.delegateTarget=this;if(t.preDispatch&&t.preDispatch.call(this,c)===!1)return;if(q&&(!c.button||c.type!=="click"))for(f=c.target;f!=this;f=f.parentNode||this)if(f.disabled!==!0||c.type!=="click"){h={},j=[];for(d=0;d<q;d++)l=o[d],m=l.selector,h[m]===b&&(h[m]=l.needsContext?p(m,this).index(f)>=0:p.find(m,this,null,[f]).length),h[m]&&j.push(l);j.length&&u.push({elem:f,matches:j})}o.length>q&&u.push({elem:this,matches:o.slice(q)});for(d=0;d<u.length&&!c.isPropagationStopped();d++){i=u[d],c.currentTarget=i.elem;for(e=0;e<i.matches.length&&!c.isImmediatePropagationStopped();e++){l=i.matches[e];if(s||!c.namespace&&!l.namespace||c.namespac
 e_re&&c.namespace_re.test(l.namespace))c.data=l.data,c.handleObj=l,g=((p.event.special[l.origType]||{}).handle||l.handler).apply(i.elem,r),g!==b&&(c.result=g,g===!1&&(c.preventDefault(),c.stopPropagation()))}}return t.postDispatch&&t.postDispatch.call(this,c),c.result},props:"attrChange attrName relatedNode srcElement altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(a,b){return a.which==null&&(a.which=b.charCode!=null?b.charCode:b.keyCode),a}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(a,c){var d,f,g,h=c.button,i=c.fromElement;return a.pageX==null&&c.clientX!=null&&(d=a.target.ownerDocument||e,f=d.documentElement,g=d.body,a.pageX=c.clientX+(f&&f.scrollLeft||g&&g.scrollLeft||0)-(f&&f.clientLeft||g&&g.clientLeft||0),a.pageY=
 c.clientY+(f&&f.scrollTop||g&&g.scrollTop||0)-(f&&f.clientTop||g&&g.clientTop||0)),!a.relatedTarget&&i&&(a.relatedTarget=i===a.target?c.toElement:i),!a.which&&h!==b&&(a.which=h&1?1:h&2?3:h&4?2:0),a}},fix:function(a){if(a[p.expando])return a;var b,c,d=a,f=p.event.fixHooks[a.type]||{},g=f.props?this.props.concat(f.props):this.props;a=p.Event(d);for(b=g.length;b;)c=g[--b],a[c]=d[c];return a.target||(a.target=d.srcElement||e),a.target.nodeType===3&&(a.target=a.target.parentNode),a.metaKey=!!a.metaKey,f.filter?f.filter(a,d):a},special:{load:{noBubble:!0},focus:{delegateType:"focusin"},blur:{delegateType:"focusout"},beforeunload:{setup:function(a,b,c){p.isWindow(this)&&(this.onbeforeunload=c)},teardown:function(a,b){this.onbeforeunload===b&&(this.onbeforeunload=null)}}},simulate:function(a,b,c,d){var e=p.extend(new p.Event,c,{type:a,isSimulated:!0,originalEvent:{}});d?p.event.trigger(e,null,b):p.event.dispatch.call(b,e),e.isDefaultPrevented()&&c.preventDefault()}},p.event.handle=p.event.d
 ispatch,p.removeEvent=e.removeEventListener?function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c,!1)}:function(a,b,c){var d="on"+b;a.detachEvent&&(typeof a[d]=="undefined"&&(a[d]=null),a.detachEvent(d,c))},p.Event=function(a,b){if(this instanceof p.Event)a&&a.type?(this.originalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||a.returnValue===!1||a.getPreventDefault&&a.getPreventDefault()?bb:ba):this.type=a,b&&p.extend(this,b),this.timeStamp=a&&a.timeStamp||p.now(),this[p.expando]=!0;else return new p.Event(a,b)},p.Event.prototype={preventDefault:function(){this.isDefaultPrevented=bb;var a=this.originalEvent;if(!a)return;a.preventDefault?a.preventDefault():a.returnValue=!1},stopPropagation:function(){this.isPropagationStopped=bb;var a=this.originalEvent;if(!a)return;a.stopPropagation&&a.stopPropagation(),a.cancelBubble=!0},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=bb,this.stopPropagation()},isDefaultPrevented:ba,isPropagati
 onStopped:ba,isImmediatePropagationStopped:ba},p.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(a,b){p.event.special[a]={delegateType:b,bindType:b,handle:function(a){var c,d=this,e=a.relatedTarget,f=a.handleObj,g=f.selector;if(!e||e!==d&&!p.contains(d,e))a.type=f.origType,c=f.handler.apply(this,arguments),a.type=b;return c}}}),p.support.submitBubbles||(p.event.special.submit={setup:function(){if(p.nodeName(this,"form"))return!1;p.event.add(this,"click._submit keypress._submit",function(a){var c=a.target,d=p.nodeName(c,"input")||p.nodeName(c,"button")?c.form:b;d&&!p._data(d,"_submit_attached")&&(p.event.add(d,"submit._submit",function(a){a._submit_bubble=!0}),p._data(d,"_submit_attached",!0))})},postDispatch:function(a){a._submit_bubble&&(delete a._submit_bubble,this.parentNode&&!a.isTrigger&&p.event.simulate("submit",this.parentNode,a,!0))},teardown:function(){if(p.nodeName(this,"form"))return!1;p.event.remove(this,"._submit")}}),p.support.changeBubbles||(p.event.speci
 al.change={setup:function(){if(V.test(this.nodeName)){if(this.type==="checkbox"||this.type==="radio")p.event.add(this,"propertychange._change",function(a){a.originalEvent.propertyName==="checked"&&(this._just_changed=!0)}),p.event.add(this,"click._change",function(a){this._just_changed&&!a.isTrigger&&(this._just_changed=!1),p.event.simulate("change",this,a,!0)});return!1}p.event.add(this,"beforeactivate._change",function(a){var b=a.target;V.test(b.nodeName)&&!p._data(b,"_change_attached")&&(p.event.add(b,"change._change",function(a){this.parentNode&&!a.isSimulated&&!a.isTrigger&&p.event.simulate("change",this.parentNode,a,!0)}),p._data(b,"_change_attached",!0))})},handle:function(a){var b=a.target;if(this!==b||a.isSimulated||a.isTrigger||b.type!=="radio"&&b.type!=="checkbox")return a.handleObj.handler.apply(this,arguments)},teardown:function(){return p.event.remove(this,"._change"),!V.test(this.nodeName)}}),p.support.focusinBubbles||p.each({focus:"focusin",blur:"focusout"},function(
 a,b){var c=0,d=function(a){p.event.simulate(b,a.target,p.event.fix(a),!0)};p.event.special[b]={setup:function(){c++===0&&e.addEventListener(a,d,!0)},teardown:function(){--c===0&&e.removeEventListener(a,d,!0)}}}),p.fn.extend({on:function(a,c,d,e,f){var g,h;if(typeof a=="object"){typeof c!="string"&&(d=d||c,c=b);for(h in a)this.on(h,c,d,a[h],f);return this}d==null&&e==null?(e=c,d=c=b):e==null&&(typeof c=="string"?(e=d,d=b):(e=d,d=c,c=b));if(e===!1)e=ba;else if(!e)return this;return f===1&&(g=e,e=function(a){return p().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=p.guid++)),this.each(function(){p.event.add(this,a,e,d,c)})},one:function(a,b,c,d){return this.on(a,b,c,d,1)},off:function(a,c,d){var e,f;if(a&&a.preventDefault&&a.handleObj)return e=a.handleObj,p(a.delegateTarget).off(e.namespace?e.origType+"."+e.namespace:e.origType,e.selector,e.handler),this;if(typeof a=="object"){for(f in a)this.off(f,c,a[f]);return this}if(c===!1||typeof c=="function")d=c,c=b;return d===!1&&(d=b
 a),this.each(function(){p.event.remove(this,a,d,c)})},bind:function(a,b,c){return this.on(a,null,b,c)},unbind:function(a,b){return this.off(a,null,b)},live:function(a,b,c){return p(this.context).on(a,this.selector,b,c),this},die:function(a,b){return p(this.context).off(a,this.selector||"**",b),this},delegate:function(a,b,c,d){return this.on(b,a,c,d)},undelegate:function(a,b,c){return arguments.length===1?this.off(a,"**"):this.off(b,a||"**",c)},trigger:function(a,b){return this.each(function(){p.event.trigger(a,b,this)})},triggerHandler:function(a,b){if(this[0])return p.event.trigger(a,b,this[0],!0)},toggle:function(a){var b=arguments,c=a.guid||p.guid++,d=0,e=function(c){var e=(p._data(this,"lastToggle"+a.guid)||0)%d;return p._data(this,"lastToggle"+a.guid,e+1),c.preventDefault(),b[e].apply(this,arguments)||!1};e.guid=c;while(d<b.length)b[d++].guid=c;return this.click(e)},hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}}),p.each("blur focus focusin focusout load resize
  scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(a,b){p.fn[b]=function(a,c){return c==null&&(c=a,a=null),arguments.length>0?this.on(b,null,a,c):this.trigger(b)},Y.test(b)&&(p.event.fixHooks[b]=p.event.keyHooks),Z.test(b)&&(p.event.fixHooks[b]=p.event.mouseHooks)}),function(a,b){function bc(a,b,c,d){c=c||[],b=b||r;var e,f,i,j,k=b.nodeType;if(!a||typeof a!="string")return c;if(k!==1&&k!==9)return[];i=g(b);if(!i&&!d)if(e=P.exec(a))if(j=e[1]){if(k===9){f=b.getElementById(j);if(!f||!f.parentNode)return c;if(f.id===j)return c.push(f),c}else if(b.ownerDocument&&(f=b.ownerDocument.getElementById(j))&&h(b,f)&&f.id===j)return c.push(f),c}else{if(e[2])return w.apply(c,x.call(b.getElementsByTagName(a),0)),c;if((j=e[3])&&_&&b.getElementsByClassName)return w.apply(c,x.call(b.getElementsByClassName(j),0)),c}return bp(a.replace(L,"$1"),b,c,d,i)}function bd(a){return f
 unction(b){var c=b.nodeName.toLowerCase();return c==="input"&&b.type===a}}function be(a){return function(b){var c=b.nodeName.toLowerCase();return(c==="input"||c==="button")&&b.type===a}}function bf(a){return z(function(b){return b=+b,z(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function bg(a,b,c){if(a===b)return c;var d=a.nextSibling;while(d){if(d===b)return-1;d=d.nextSibling}return 1}function bh(a,b){var c,d,f,g,h,i,j,k=C[o][a];if(k)return b?0:k.slice(0);h=a,i=[],j=e.preFilter;while(h){if(!c||(d=M.exec(h)))d&&(h=h.slice(d[0].length)),i.push(f=[]);c=!1;if(d=N.exec(h))f.push(c=new q(d.shift())),h=h.slice(c.length),c.type=d[0].replace(L," ");for(g in e.filter)(d=W[g].exec(h))&&(!j[g]||(d=j[g](d,r,!0)))&&(f.push(c=new q(d.shift())),h=h.slice(c.length),c.type=g,c.matches=d);if(!c)break}return b?h.length:h?bc.error(a):C(a,i).slice(0)}function bi(a,b,d){var e=b.dir,f=d&&b.dir==="parentNode",g=u++;return b.first?function(b,c,d){while(b=b[
 e])if(f||b.nodeType===1)return a(b,c,d)}:function(b,d,h){if(!h){var i,j=t+" "+g+" ",k=j+c;while(b=b[e])if(f||b.nodeType===1){if((i=b[o])===k)return b.sizset;if(typeof i=="string"&&i.indexOf(j)===0){if(b.sizset)return b}else{b[o]=k;if(a(b,d,h))return b.sizset=!0,b;b.sizset=!1}}}else while(b=b[e])if(f||b.nodeType===1)if(a(b,d,h))return b}}function bj(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function bk(a,b,c,d,e){var f,g=[],h=0,i=a.length,j=b!=null;for(;h<i;h++)if(f=a[h])if(!c||c(f,d,e))g.push(f),j&&b.push(h);return g}function bl(a,b,c,d,e,f){return d&&!d[o]&&(d=bl(d)),e&&!e[o]&&(e=bl(e,f)),z(function(f,g,h,i){if(f&&e)return;var j,k,l,m=[],n=[],o=g.length,p=f||bo(b||"*",h.nodeType?[h]:h,[],f),q=a&&(f||!b)?bk(p,m,a,h,i):p,r=c?e||(f?a:o||d)?[]:g:q;c&&c(q,r,h,i);if(d){l=bk(r,n),d(l,[],h,i),j=l.length;while(j--)if(k=l[j])r[n[j]]=!(q[n[j]]=k)}if(f){j=a&&r.length;while(j--)if(k=r[j])f[m[j]]=!(g[m[j]]=k)}else r=bk(r===g?r.splice(o,
 r.length):r),e?e(null,g,r,i):w.apply(g,r)})}function bm(a){var b,c,d,f=a.length,g=e.relative[a[0].type],h=g||e.relative[" "],i=g?1:0,j=bi(function(a){return a===b},h,!0),k=bi(function(a){return y.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==l)||((b=c).nodeType?j(a,c,d):k(a,c,d))}];for(;i<f;i++)if(c=e.relative[a[i].type])m=[bi(bj(m),c)];else{c=e.filter[a[i].type].apply(null,a[i].matches);if(c[o]){d=++i;for(;d<f;d++)if(e.relative[a[d].type])break;return bl(i>1&&bj(m),i>1&&a.slice(0,i-1).join("").replace(L,"$1"),c,i<d&&bm(a.slice(i,d)),d<f&&bm(a=a.slice(d)),d<f&&a.join(""))}m.push(c)}return bj(m)}function bn(a,b){var d=b.length>0,f=a.length>0,g=function(h,i,j,k,m){var n,o,p,q=[],s=0,u="0",x=h&&[],y=m!=null,z=l,A=h||f&&e.find.TAG("*",m&&i.parentNode||i),B=t+=z==null?1:Math.E;y&&(l=i!==r&&i,c=g.el);for(;(n=A[u])!=null;u++){if(f&&n){for(o=0;p=a[o];o++)if(p(n,i,j)){k.push(n);break}y&&(t=B,c=++g.el)}d&&((n=!p&&n)&&s--,h&&x.push(n))}s+=u;if(d&&u!==s){for(o=0;p=b[o];o++)p(x,q,i,j)
 ;if(h){if(s>0)while(u--)!x[u]&&!q[u]&&(q[u]=v.call(k));q=bk(q)}w.apply(k,q),y&&!h&&q.length>0&&s+b.length>1&&bc.uniqueSort(k)}return y&&(t=B,l=z),x};return g.el=0,d?z(g):g}function bo(a,b,c,d){var e=0,f=b.length;for(;e<f;e++)bc(a,b[e],c,d);return c}function bp(a,b,c,d,f){var g,h,j,k,l,m=bh(a),n=m.length;if(!d&&m.length===1){h=m[0]=m[0].slice(0);if(h.length>2&&(j=h[0]).type==="ID"&&b.nodeType===9&&!f&&e.relative[h[1].type]){b=e.find.ID(j.matches[0].replace(V,""),b,f)[0];if(!b)return c;a=a.slice(h.shift().length)}for(g=W.POS.test(a)?-1:h.length-1;g>=0;g--){j=h[g];if(e.relative[k=j.type])break;if(l=e.find[k])if(d=l(j.matches[0].replace(V,""),R.test(h[0].type)&&b.parentNode||b,f)){h.splice(g,1),a=d.length&&h.join("");if(!a)return w.apply(c,x.call(d,0)),c;break}}}return i(a,m)(d,b,f,c,R.test(a)),c}function bq(){}var c,d,e,f,g,h,i,j,k,l,m=!0,n="undefined",o=("sizcache"+Math.random()).replace(".",""),q=String,r=a.document,s=r.documentElement,t=0,u=0,v=[].pop,w=[].push,x=[].slice,y=[].index
 Of||function(a){var b=0,c=this.length;for(;b<c;b++)if(this[b]===a)return b;return-1},z=function(a,b){return a[o]=b==null||b,a},A=function(){var a={},b=[];return z(function(c,d){return b.push(c)>e.cacheLength&&delete a[b.shift()],a[c]=d},a)},B=A(),C=A(),D=A(),E="[\\x20\\t\\r\\n\\f]",F="(?:\\\\.|[-\\w]|[^\\x00-\\xa0])+",G=F.replace("w","w#"),H="([*^$|!~]?=)",I="\\["+E+"*("+F+")"+E+"*(?:"+H+E+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+G+")|)|)"+E+"*\\]",J=":("+F+")(?:\\((?:(['\"])((?:\\\\.|[^\\\\])*?)\\2|([^()[\\]]*|(?:(?:"+I+")|[^:]|\\\\.)*|.*))\\)|)",K=":(even|odd|eq|gt|lt|nth|first|last)(?:\\("+E+"*((?:-\\d)?\\d*)"+E+"*\\)|)(?=[^-]|$)",L=new RegExp("^"+E+"+|((?:^|[^\\\\])(?:\\\\.)*)"+E+"+$","g"),M=new RegExp("^"+E+"*,"+E+"*"),N=new RegExp("^"+E+"*([\\x20\\t\\r\\n\\f>+~])"+E+"*"),O=new RegExp(J),P=/^(?:#([\w\-]+)|(\w+)|\.([\w\-]+))$/,Q=/^:not/,R=/[\x20\t\r\n\f]*[+~]/,S=/:not\($/,T=/h\d/i,U=/input|select|textarea|button/i,V=/\\(?!\\)/g,W={ID:new RegExp("^#("+F+")"),CLASS:new RegExp("^\\.
 ("+F+")"),NAME:new RegExp("^\\[name=['\"]?("+F+")['\"]?\\]"),TAG:new RegExp("^("+F.replace("w","w*")+")"),ATTR:new RegExp("^"+I),PSEUDO:new RegExp("^"+J),POS:new RegExp(K,"i"),CHILD:new RegExp("^:(only|nth|first|last)-child(?:\\("+E+"*(even|odd|(([+-]|)(\\d*)n|)"+E+"*(?:([+-]|)"+E+"*(\\d+)|))"+E+"*\\)|)","i"),needsContext:new RegExp("^"+E+"*[>+~]|"+K,"i")},X=function(a){var b=r.createElement("div");try{return a(b)}catch(c){return!1}finally{b=null}},Y=X(function(a){return a.appendChild(r.createComment("")),!a.getElementsByTagName("*").length}),Z=X(function(a){return a.innerHTML="<a href='#'></a>",a.firstChild&&typeof a.firstChild.getAttribute!==n&&a.firstChild.getAttribute("href")==="#"}),$=X(function(a){a.innerHTML="<select></select>";var b=typeof a.lastChild.getAttribute("multiple");return b!=="boolean"&&b!=="string"}),_=X(function(a){return a.innerHTML="<div class='hidden e'></div><div class='hidden'></div>",!a.getElementsByClassName||!a.getElementsByClassName("e").length?!1:(a.la
 stChild.className="e",a.getElementsByClassName("e").length===2)}),ba=X(function(a){a.id=o+0,a.innerHTML="<a name='"+o+"'></a><div name='"+o+"'></div>",s.insertBefore(a,s.firstChild);var b=r.getElementsByName&&r.getElementsByName(o).length===2+r.getElementsByName(o+0).length;return d=!r.getElementById(o),s.removeChild(a),b});try{x.call(s.childNodes,0)[0].nodeType}catch(bb){x=function(a){var b,c=[];for(;b=this[a];a++)c.push(b);return c}}bc.matches=function(a,b){return bc(a,null,null,b)},bc.matchesSelector=function(a,b){return bc(b,null,null,[a]).length>0},f=bc.getText=function(a){var b,c="",d=0,e=a.nodeType;if(e){if(e===1||e===9||e===11){if(typeof a.textContent=="string")return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=f(a)}else if(e===3||e===4)return a.nodeValue}else for(;b=a[d];d++)c+=f(b);return c},g=bc.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?b.nodeName!=="HTML":!1},h=bc.contains=s.contains?function(a,b){var c=a.nodeType===9?a.documentEl
 ement:a,d=b&&b.parentNode;return a===d||!!(d&&d.nodeType===1&&c.contains&&c.contains(d))}:s.compareDocumentPosition?function(a,b){return b&&!!(a.compareDocumentPosition(b)&16)}:function(a,b){while(b=b.parentNode)if(b===a)return!0;return!1},bc.attr=function(a,b){var c,d=g(a);return d||(b=b.toLowerCase()),(c=e.attrHandle[b])?c(a):d||$?a.getAttribute(b):(c=a.getAttributeNode(b),c?typeof a[b]=="boolean"?a[b]?b:null:c.specified?c.value:null:null)},e=bc.selectors={cacheLength:50,createPseudo:z,match:W,attrHandle:Z?{}:{href:function(a){return a.getAttribute("href",2)},type:function(a){return a.getAttribute("type")}},find:{ID:d?function(a,b,c){if(typeof b.getElementById!==n&&!c){var d=b.getElementById(a);return d&&d.parentNode?[d]:[]}}:function(a,c,d){if(typeof c.getElementById!==n&&!d){var e=c.getElementById(a);return e?e.id===a||typeof e.getAttributeNode!==n&&e.getAttributeNode("id").value===a?[e]:b:[]}},TAG:Y?function(a,b){if(typeof b.getElementsByTagName!==n)return b.getElementsByTagNam
 e(a)}:function(a,b){var c=b.getElementsByTagName(a);if(a==="*"){var d,e=[],f=0;for(;d=c[f];f++)d.nodeType===1&&e.push(d);return e}return c},NAME:ba&&function(a,b){if(typeof b.getElementsByName!==n)return b.getElementsByName(name)},CLASS:_&&function(a,b,c){if(typeof b.getElementsByClassName!==n&&!c)return b.getElementsByClassName(a)}},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(V,""),a[3]=(a[4]||a[5]||"").replace(V,""),a[2]==="~="&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),a[1]==="nth"?(a[2]||bc.error(a[0]),a[3]=+(a[3]?a[4]+(a[5]||1):2*(a[2]==="even"||a[2]==="odd")),a[4]=+(a[6]+a[7]||a[2]==="odd")):a[2]&&bc.error(a[0]),a},PSEUDO:function(a){var b,c;if(W.CHILD.test(a[0]))return null;if(a[3])a[2]=a[3];else if(b=a[4])O.test(b)&&(c=bh(b,!0))&&(c=b.indexOf(")",b.length-c)-b.length)&&(b=b.slice(0,c),a[0]=a[0].sl
 ice(0,c)),a[2]=b;return a.slice(0,3)}},filter:{ID:d?function(a){return a=a.replace(V,""),function(b){return b.getAttribute("id")===a}}:function(a){return a=a.replace(V,""),function(b){var c=typeof b.getAttributeNode!==n&&b.getAttributeNode("id");return c&&c.value===a}},TAG:function(a){return a==="*"?function(){return!0}:(a=a.replace(V,"").toLowerCase(),function(b){return b.nodeName&&b.nodeName.toLowerCase()===a})},CLASS:function(a){var b=B[o][a];return b||(b=B(a,new RegExp("(^|"+E+")"+a+"("+E+"|$)"))),function(a){return b.test(a.className||typeof a.getAttribute!==n&&a.getAttribute("class")||"")}},ATTR:function(a,b,c){return function(d,e){var f=bc.attr(d,a);return f==null?b==="!=":b?(f+="",b==="="?f===c:b==="!="?f!==c:b==="^="?c&&f.indexOf(c)===0:b==="*="?c&&f.indexOf(c)>-1:b==="$="?c&&f.substr(f.length-c.length)===c:b==="~="?(" "+f+" ").indexOf(c)>-1:b==="|="?f===c||f.substr(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d){return a==="nth"?function(a){var b,e,f=a.parentNode;if
 (c===1&&d===0)return!0;if(f){e=0;for(b=f.firstChild;b;b=b.nextSibling)if(b.nodeType===1){e++;if(a===b)break}}return e-=d,e===c||e%c===0&&e/c>=0}:function(b){var c=b;switch(a){case"only":case"first":while(c=c.previousSibling)if(c.nodeType===1)return!1;if(a==="first")return!0;c=b;case"last":while(c=c.nextSibling)if(c.nodeType===1)return!1;return!0}}},PSEUDO:function(a,b){var c,d=e.pseudos[a]||e.setFilters[a.toLowerCase()]||bc.error("unsupported pseudo: "+a);return d[o]?d(b):d.length>1?(c=[a,a,"",b],e.setFilters.hasOwnProperty(a.toLowerCase())?z(function(a,c){var e,f=d(a,b),g=f.length;while(g--)e=y.call(a,f[g]),a[e]=!(c[e]=f[g])}):function(a){return d(a,0,c)}):d}},pseudos:{not:z(function(a){var b=[],c=[],d=i(a.replace(L,"$1"));return d[o]?z(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)if(f=g[h])a[h]=!(b[h]=f)}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:z(function(a){return function(b){return bc(a,b).length>0}}),contains:z(function(a){return function(b
 ){return(b.textContent||b.innerText||f(b)).indexOf(a)>-1}}),enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&!!a.checked||b==="option"&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},parent:function(a){return!e.pseudos.empty(a)},empty:function(a){var b;a=a.firstChild;while(a){if(a.nodeName>"@"||(b=a.nodeType)===3||b===4)return!1;a=a.nextSibling}return!0},header:function(a){return T.test(a.nodeName)},text:function(a){var b,c;return a.nodeName.toLowerCase()==="input"&&(b=a.type)==="text"&&((c=a.getAttribute("type"))==null||c.toLowerCase()===b)},radio:bd("radio"),checkbox:bd("checkbox"),file:bd("file"),password:bd("password"),image:bd("image"),submit:be("submit"),reset:be("reset"),button:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&a.type==="button"||b==="button"},input:function(a){return U.test(a.nodeName)
 },focus:function(a){var b=a.ownerDocument;return a===b.activeElement&&(!b.hasFocus||b.hasFocus())&&(!!a.type||!!a.href)},active:function(a){return a===a.ownerDocument.activeElement},first:bf(function(a,b,c){return[0]}),last:bf(function(a,b,c){return[b-1]}),eq:bf(function(a,b,c){return[c<0?c+b:c]}),even:bf(function(a,b,c){for(var d=0;d<b;d+=2)a.push(d);return a}),odd:bf(function(a,b,c){for(var d=1;d<b;d+=2)a.push(d);return a}),lt:bf(function(a,b,c){for(var d=c<0?c+b:c;--d>=0;)a.push(d);return a}),gt:bf(function(a,b,c){for(var d=c<0?c+b:c;++d<b;)a.push(d);return a})}},j=s.compareDocumentPosition?function(a,b){return a===b?(k=!0,0):(!a.compareDocumentPosition||!b.compareDocumentPosition?a.compareDocumentPosition:a.compareDocumentPosition(b)&4)?-1:1}:function(a,b){if(a===b)return k=!0,0;if(a.sourceIndex&&b.sourceIndex)return a.sourceIndex-b.sourceIndex;var c,d,e=[],f=[],g=a.parentNode,h=b.parentNode,i=g;if(g===h)return bg(a,b);if(!g)return-1;if(!h)return 1;while(i)e.unshift(i),i=i.paren
 tNode;i=h;while(i)f.unshift(i),i=i.parentNode;c=e.length,d=f.length;for(var j=0;j<c&&j<d;j++)if(e[j]!==f[j])return bg(e[j],f[j]);return j===c?bg(a,f[j],-1):bg(e[j],b,1)},[0,0].sort(j),m=!k,bc.uniqueSort=function(a){var b,c=1;k=m,a.sort(j);if(k)for(;b=a[c];c++)b===a[c-1]&&a.splice(c--,1);return a},bc.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},i=bc.compile=function(a,b){var c,d=[],e=[],f=D[o][a];if(!f){b||(b=bh(a)),c=b.length;while(c--)f=bm(b[c]),f[o]?d.push(f):e.push(f);f=D(a,bn(e,d))}return f},r.querySelectorAll&&function(){var a,b=bp,c=/'|\\/g,d=/\=[\x20\t\r\n\f]*([^'"\]]*)[\x20\t\r\n\f]*\]/g,e=[":focus"],f=[":active",":focus"],h=s.matchesSelector||s.mozMatchesSelector||s.webkitMatchesSelector||s.oMatchesSelector||s.msMatchesSelector;X(function(a){a.innerHTML="<select><option selected=''></option></select>",a.querySelectorAll("[selected]").length||e.push("\\["+E+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),a.querySelectorAll("
 :checked").length||e.push(":checked")}),X(function(a){a.innerHTML="<p test=''></p>",a.querySelectorAll("[test^='']").length&&e.push("[*^$]="+E+"*(?:\"\"|'')"),a.innerHTML="<input type='hidden'/>",a.querySelectorAll(":enabled").length||e.push(":enabled",":disabled")}),e=new RegExp(e.join("|")),bp=function(a,d,f,g,h){if(!g&&!h&&(!e||!e.test(a))){var i,j,k=!0,l=o,m=d,n=d.nodeType===9&&a;if(d.nodeType===1&&d.nodeName.toLowerCase()!=="object"){i=bh(a),(k=d.getAttribute("id"))?l=k.replace(c,"\\$&"):d.setAttribute("id",l),l="[id='"+l+"'] ",j=i.length;while(j--)i[j]=l+i[j].join("");m=R.test(a)&&d.parentNode||d,n=i.join(",")}if(n)try{return w.apply(f,x.call(m.querySelectorAll(n),0)),f}catch(p){}finally{k||d.removeAttribute("id")}}return b(a,d,f,g,h)},h&&(X(function(b){a=h.call(b,"div");try{h.call(b,"[test!='']:sizzle"),f.push("!=",J)}catch(c){}}),f=new RegExp(f.join("|")),bc.matchesSelector=function(b,c){c=c.replace(d,"='$1']");if(!g(b)&&!f.test(c)&&(!e||!e.test(c)))try{var i=h.call(b,c);if(
 i||a||b.document&&b.document.nodeType!==11)return i}catch(j){}return bc(c,null,null,[b]).length>0})}(),e.pseudos.nth=e.pseudos.eq,e.filters=bq.prototype=e.pseudos,e.setFilters=new bq,bc.attr=p.attr,p.find=bc,p.expr=bc.selectors,p.expr[":"]=p.expr.pseudos,p.unique=bc.uniqueSort,p.text=bc.getText,p.isXMLDoc=bc.isXML,p.contains=bc.contains}(a);var bc=/Until$/,bd=/^(?:parents|prev(?:Until|All))/,be=/^.[^:#\[\.,]*$/,bf=p.expr.match.needsContext,bg={children:!0,contents:!0,next:!0,prev:!0};p.fn.extend({find:function(a){var b,c,d,e,f,g,h=this;if(typeof a!="string")return p(a).filter(function(){for(b=0,c=h.length;b<c;b++)if(p.contains(h[b],this))return!0});g=this.pushStack("","find",a);for(b=0,c=this.length;b<c;b++){d=g.length,p.find(a,this[b],g);if(b>0)for(e=d;e<g.length;e++)for(f=0;f<d;f++)if(g[f]===g[e]){g.splice(e--,1);break}}return g},has:function(a){var b,c=p(a,this),d=c.length;return this.filter(function(){for(b=0;b<d;b++)if(p.contains(this,c[b]))return!0})},not:function(a){return th
 is.pushStack(bj(this,a,!1),"not",a)},filter:function(a){return this.pushStack(bj(this,a,!0),"filter",a)},is:function(a){return!!a&&(typeof a=="string"?bf.test(a)?p(a,this.context).index(this[0])>=0:p.filter(a,this).length>0:this.filter(a).length>0)},closest:function(a,b){var c,d=0,e=this.length,f=[],g=bf.test(a)||typeof a!="string"?p(a,b||this.context):0;for(;d<e;d++){c=this[d];while(c&&c.ownerDocument&&c!==b&&c.nodeType!==11){if(g?g.index(c)>-1:p.find.matchesSelector(c,a)){f.push(c);break}c=c.parentNode}}return f=f.length>1?p.unique(f):f,this.pushStack(f,"closest",a)},index:function(a){return a?typeof a=="string"?p.inArray(this[0],p(a)):p.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.prevAll().length:-1},add:function(a,b){var c=typeof a=="string"?p(a,b):p.makeArray(a&&a.nodeType?[a]:a),d=p.merge(this.get(),c);return this.pushStack(bh(c[0])||bh(d[0])?d:p.unique(d))},addBack:function(a){return this.add(a==null?this.prevObject:this.prevObject.filter(a))}}),p.fn.andSel
 f=p.fn.addBack,p.each({parent:function(a){var b=a.parentNode;return b&&b.nodeType!==11?b:null},parents:function(a){return p.dir(a,"parentNode")},parentsUntil:function(a,b,c){return p.dir(a,"parentNode",c)},next:function(a){return bi(a,"nextSibling")},prev:function(a){return bi(a,"previousSibling")},nextAll:function(a){return p.dir(a,"nextSibling")},prevAll:function(a){return p.dir(a,"previousSibling")},nextUntil:function(a,b,c){return p.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return p.dir(a,"previousSibling",c)},siblings:function(a){return p.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return p.sibling(a.firstChild)},contents:function(a){return p.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:p.merge([],a.childNodes)}},function(a,b){p.fn[a]=function(c,d){var e=p.map(this,b,c);return bc.test(a)||(d=c),d&&typeof d=="string"&&(e=p.filter(d,e)),e=this.length>1&&!bg[a]?p.unique(e):e,this.length>1&&bd.test(a)&&(e=e.reverse()),this.pushStack(e,
 a,k.call(arguments).join(","))}}),p.extend({filter:function(a,b,c){return c&&(a=":not("+a+")"),b.length===1?p.find.matchesSelector(b[0],a)?[b[0]]:[]:p.find.matches(a,b)},dir:function(a,c,d){var e=[],f=a[c];while(f&&f.nodeType!==9&&(d===b||f.nodeType!==1||!p(f).is(d)))f.nodeType===1&&e.push(f),f=f[c];return e},sibling:function(a,b){var c=[];for(;a;a=a.nextSibling)a.nodeType===1&&a!==b&&c.push(a);return c}});var bl="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",bm=/ jQuery\d+="(?:null|\d+)"/g,bn=/^\s+/,bo=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bp=/<([\w:]+)/,bq=/<tbody/i,br=/<|&#?\w+;/,bs=/<(?:script|style|link)/i,bt=/<(?:script|object|embed|option|style)/i,bu=new RegExp("<(?:"+bl+")[\\s/>]","i"),bv=/^(?:checkbox|radio)$/,bw=/checked\s*(?:[^=]|=\s*.checked.)/i,bx=/\/(java|ecma)script/i,by=/^\s*<!(?:\[CDATA\[|\-\-)|[\]\-]{2}>\s*$/g,bz={option:
 [1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],area:[1,"<map>","</map>"],_default:[0,"",""]},bA=bk(e),bB=bA.appendChild(e.createElement("div"));bz.optgroup=bz.option,bz.tbody=bz.tfoot=bz.colgroup=bz.caption=bz.thead,bz.th=bz.td,p.support.htmlSerialize||(bz._default=[1,"X<div>","</div>"]),p.fn.extend({text:function(a){return p.access(this,function(a){return a===b?p.text(this):this.empty().append((this[0]&&this[0].ownerDocument||e).createTextNode(a))},null,a,arguments.length)},wrapAll:function(a){if(p.isFunction(a))return this.each(function(b){p(this).wrapAll(a.call(this,b))});if(this[0]){var b=p(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&a.firstChild.nodeType===1)a=a.fir
 stChild;return a}).append(this)}return this},wrapInner:function(a){return p.isFunction(a)?this.each(function(b){p(this).wrapInner(a.call(this,b))}):this.each(function(){var b=p(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=p.isFunction(a);return this.each(function(c){p(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(){return this.parent().each(function(){p.nodeName(this,"body")||p(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.appendChild(a)})},prepend:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.insertBefore(a,this.firstChild)})},before:function(){if(!bh(this[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(a,this),"before",this.selector)}},after:function(){if(!bh(th
 is[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this.nextSibling)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(this,a),"after",this.selector)}},remove:function(a,b){var c,d=0;for(;(c=this[d])!=null;d++)if(!a||p.filter(a,[c]).length)!b&&c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),p.cleanData([c])),c.parentNode&&c.parentNode.removeChild(c);return this},empty:function(){var a,b=0;for(;(a=this[b])!=null;b++){a.nodeType===1&&p.cleanData(a.getElementsByTagName("*"));while(a.firstChild)a.removeChild(a.firstChild)}return this},clone:function(a,b){return a=a==null?!1:a,b=b==null?a:b,this.map(function(){return p.clone(this,a,b)})},html:function(a){return p.access(this,function(a){var c=this[0]||{},d=0,e=this.length;if(a===b)return c.nodeType===1?c.innerHTML.replace(bm,""):b;if(typeof a=="string"&&!bs.test(a)&&(p.support.htmlSerialize||!bu.test(a))&&(p.support.leadingWhitespace||!bn.test(a))&&!bz[(bp.exec(a)||["
 ",""])[1].toLowerCase()]){a=a.replace(bo,"<$1></$2>");try{for(;d<e;d++)c=this[d]||{},c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),c.innerHTML=a);c=0}catch(f){}}c&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(a){return bh(this[0])?this.length?this.pushStack(p(p.isFunction(a)?a():a),"replaceWith",a):this:p.isFunction(a)?this.each(function(b){var c=p(this),d=c.html();c.replaceWith(a.call(this,b,d))}):(typeof a!="string"&&(a=p(a).detach()),this.each(function(){var b=this.nextSibling,c=this.parentNode;p(this).remove(),b?p(b).before(a):p(c).append(a)}))},detach:function(a){return this.remove(a,!0)},domManip:function(a,c,d){a=[].concat.apply([],a);var e,f,g,h,i=0,j=a[0],k=[],l=this.length;if(!p.support.checkClone&&l>1&&typeof j=="string"&&bw.test(j))return this.each(function(){p(this).domManip(a,c,d)});if(p.isFunction(j))return this.each(function(e){var f=p(this);a[0]=j.call(this,e,c?f.html():b),f.domManip(a,c,d)});if(this[0]){e=p.buildFragment(a,this
 ,k),g=e.fragment,f=g.firstChild,g.childNodes.length===1&&(g=f);if(f){c=c&&p.nodeName(f,"tr");for(h=e.cacheable||l-1;i<l;i++)d.call(c&&p.nodeName(this[i],"table")?bC(this[i],"tbody"):this[i],i===h?g:p.clone(g,!0,!0))}g=f=null,k.length&&p.each(k,function(a,b){b.src?p.ajax?p.ajax({url:b.src,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0}):p.error("no ajax"):p.globalEval((b.text||b.textContent||b.innerHTML||"").replace(by,"")),b.parentNode&&b.parentNode.removeChild(b)})}return this}}),p.buildFragment=function(a,c,d){var f,g,h,i=a[0];return c=c||e,c=!c.nodeType&&c[0]||c,c=c.ownerDocument||c,a.length===1&&typeof i=="string"&&i.length<512&&c===e&&i.charAt(0)==="<"&&!bt.test(i)&&(p.support.checkClone||!bw.test(i))&&(p.support.html5Clone||!bu.test(i))&&(g=!0,f=p.fragments[i],h=f!==b),f||(f=c.createDocumentFragment(),p.clean(a,c,f,d),g&&(p.fragments[i]=h&&f)),{fragment:f,cacheable:g}},p.fragments={},p.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"
 after",replaceAll:"replaceWith"},function(a,b){p.fn[a]=function(c){var d,e=0,f=[],g=p(c),h=g.length,i=this.length===1&&this[0].parentNode;if((i==null||i&&i.nodeType===11&&i.childNodes.length===1)&&h===1)return g[b](this[0]),this;for(;e<h;e++)d=(e>0?this.clone(!0):this).get(),p(g[e])[b](d),f=f.concat(d);return this.pushStack(f,a,g.selector)}}),p.extend({clone:function(a,b,c){var d,e,f,g;p.support.html5Clone||p.isXMLDoc(a)||!bu.test("<"+a.nodeName+">")?g=a.cloneNode(!0):(bB.innerHTML=a.outerHTML,bB.removeChild(g=bB.firstChild));if((!p.support.noCloneEvent||!p.support.noCloneChecked)&&(a.nodeType===1||a.nodeType===11)&&!p.isXMLDoc(a)){bE(a,g),d=bF(a),e=bF(g);for(f=0;d[f];++f)e[f]&&bE(d[f],e[f])}if(b){bD(a,g);if(c){d=bF(a),e=bF(g);for(f=0;d[f];++f)bD(d[f],e[f])}}return d=e=null,g},clean:function(a,b,c,d){var f,g,h,i,j,k,l,m,n,o,q,r,s=b===e&&bA,t=[];if(!b||typeof b.createDocumentFragment=="undefined")b=e;for(f=0;(h=a[f])!=null;f++){typeof h=="number"&&(h+="");if(!h)continue;if(typeof h==
 "string")if(!br.test(h))h=b.createTextNode(h);else{s=s||bk(b),l=b.createElement("div"),s.appendChild(l),h=h.replace(bo,"<$1></$2>"),i=(bp.exec(h)||["",""])[1].toLowerCase(),j=bz[i]||bz._default,k=j[0],l.innerHTML=j[1]+h+j[2];while(k--)l=l.lastChild;if(!p.support.tbody){m=bq.test(h),n=i==="table"&&!m?l.firstChild&&l.firstChild.childNodes:j[1]==="<table>"&&!m?l.childNodes:[];for(g=n.length-1;g>=0;--g)p.nodeName(n[g],"tbody")&&!n[g].childNodes.length&&n[g].parentNode.removeChild(n[g])}!p.support.leadingWhitespace&&bn.test(h)&&l.insertBefore(b.createTextNode(bn.exec(h)[0]),l.firstChild),h=l.childNodes,l.parentNode.removeChild(l)}h.nodeType?t.push(h):p.merge(t,h)}l&&(h=l=s=null);if(!p.support.appendChecked)for(f=0;(h=t[f])!=null;f++)p.nodeName(h,"input")?bG(h):typeof h.getElementsByTagName!="undefined"&&p.grep(h.getElementsByTagName("input"),bG);if(c){q=function(a){if(!a.type||bx.test(a.type))return d?d.push(a.parentNode?a.parentNode.removeChild(a):a):c.appendChild(a)};for(f=0;(h=t[f])!=
 null;f++)if(!p.nodeName(h,"script")||!q(h))c.appendChild(h),typeof h.getElementsByTagName!="undefined"&&(r=p.grep(p.merge([],h.getElementsByTagName("script")),q),t.splice.apply(t,[f+1,0].concat(r)),f+=r.length)}return t},cleanData:function(a,b){var c,d,e,f,g=0,h=p.expando,i=p.cache,j=p.support.deleteExpando,k=p.event.special;for(;(e=a[g])!=null;g++)if(b||p.acceptData(e)){d=e[h],c=d&&i[d];if(c){if(c.events)for(f in c.events)k[f]?p.event.remove(e,f):p.removeEvent(e,f,c.handle);i[d]&&(delete i[d],j?delete e[h]:e.removeAttribute?e.removeAttribute(h):e[h]=null,p.deletedIds.push(d))}}}}),function(){var a,b;p.uaMatch=function(a){a=a.toLowerCase();var b=/(chrome)[ \/]([\w.]+)/.exec(a)||/(webkit)[ \/]([\w.]+)/.exec(a)||/(opera)(?:.*version|)[ \/]([\w.]+)/.exec(a)||/(msie) ([\w.]+)/.exec(a)||a.indexOf("compatible")<0&&/(mozilla)(?:.*? rv:([\w.]+)|)/.exec(a)||[];return{browser:b[1]||"",version:b[2]||"0"}},a=p.uaMatch(g.userAgent),b={},a.browser&&(b[a.browser]=!0,b.version=a.version),b.chrome?b
 .webkit=!0:b.webkit&&(b.safari=!0),p.browser=b,p.sub=function(){function a(b,c){return new a.fn.init(b,c)}p.extend(!0,a,this),a.superclass=this,a.fn=a.prototype=this(),a.fn.constructor=a,a.sub=this.sub,a.fn.init=function c(c,d){return d&&d instanceof p&&!(d instanceof a)&&(d=a(d)),p.fn.init.call(this,c,d,b)},a.fn.init.prototype=a.fn;var b=a(e);return a}}();var bH,bI,bJ,bK=/alpha\([^)]*\)/i,bL=/opacity=([^)]*)/,bM=/^(top|right|bottom|left)$/,bN=/^(none|table(?!-c[ea]).+)/,bO=/^margin/,bP=new RegExp("^("+q+")(.*)$","i"),bQ=new RegExp("^("+q+")(?!px)[a-z%]+$","i"),bR=new RegExp("^([-+])=("+q+")","i"),bS={},bT={position:"absolute",visibility:"hidden",display:"block"},bU={letterSpacing:0,fontWeight:400},bV=["Top","Right","Bottom","Left"],bW=["Webkit","O","Moz","ms"],bX=p.fn.toggle;p.fn.extend({css:function(a,c){return p.access(this,function(a,c,d){return d!==b?p.style(a,c,d):p.css(a,c)},a,c,arguments.length>1)},show:function(){return b$(this,!0)},hide:function(){return b$(this)},toggle:f
 unction(a,b){var c=typeof a=="boolean";return p.isFunction(a)&&p.isFunction(b)?bX.apply(this,arguments):this.each(function(){(c?a:bZ(this))?p(this).show():p(this).hide()})}}),p.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=bH(a,"opacity");return c===""?"1":c}}}},cssNumber:{fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":p.support.cssFloat?"cssFloat":"styleFloat"},style:function(a,c,d,e){if(!a||a.nodeType===3||a.nodeType===8||!a.style)return;var f,g,h,i=p.camelCase(c),j=a.style;c=p.cssProps[i]||(p.cssProps[i]=bY(j,i)),h=p.cssHooks[c]||p.cssHooks[i];if(d===b)return h&&"get"in h&&(f=h.get(a,!1,e))!==b?f:j[c];g=typeof d,g==="string"&&(f=bR.exec(d))&&(d=(f[1]+1)*f[2]+parseFloat(p.css(a,c)),g="number");if(d==null||g==="number"&&isNaN(d))return;g==="number"&&!p.cssNumber[i]&&(d+="px");if(!h||!("set"in h)||(d=h.set(a,d,e))!==b)try{j[c]=d}catch(k){}},css:function(a,c,d,e){var f,g,h,i=p.camelCase(c);return c=p.cssProps
 [i]||(p.cssProps[i]=bY(a.style,i)),h=p.cssHooks[c]||p.cssHooks[i],h&&"get"in h&&(f=h.get(a,!0,e)),f===b&&(f=bH(a,c)),f==="normal"&&c in bU&&(f=bU[c]),d||e!==b?(g=parseFloat(f),d||p.isNumeric(g)?g||0:f):f},swap:function(a,b,c){var d,e,f={};for(e in b)f[e]=a.style[e],a.style[e]=b[e];d=c.call(a);for(e in b)a.style[e]=f[e];return d}}),a.getComputedStyle?bH=function(b,c){var d,e,f,g,h=a.getComputedStyle(b,null),i=b.style;return h&&(d=h[c],d===""&&!p.contains(b.ownerDocument,b)&&(d=p.style(b,c)),bQ.test(d)&&bO.test(c)&&(e=i.width,f=i.minWidth,g=i.maxWidth,i.minWidth=i.maxWidth=i.width=d,d=h.width,i.width=e,i.minWidth=f,i.maxWidth=g)),d}:e.documentElement.currentStyle&&(bH=function(a,b){var c,d,e=a.currentStyle&&a.currentStyle[b],f=a.style;return e==null&&f&&f[b]&&(e=f[b]),bQ.test(e)&&!bM.test(b)&&(c=f.left,d=a.runtimeStyle&&a.runtimeStyle.left,d&&(a.runtimeStyle.left=a.currentStyle.left),f.left=b==="fontSize"?"1em":e,e=f.pixelLeft+"px",f.left=c,d&&(a.runtimeStyle.left=d)),e===""?"auto":e}
 ),p.each(["height","width"],function(a,b){p.cssHooks[b]={get:function(a,c,d){if(c)return a.offsetWidth===0&&bN.test(bH(a,"display"))?p.swap(a,bT,function(){return cb(a,b,d)}):cb(a,b,d)},set:function(a,c,d){return b_(a,c,d?ca(a,b,d,p.support.boxSizing&&p.css(a,"boxSizing")==="border-box"):0)}}}),p.support.opacity||(p.cssHooks.opacity={get:function(a,b){return bL.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle,e=p.isNumeric(b)?"alpha(opacity="+b*100+")":"",f=d&&d.filter||c.filter||"";c.zoom=1;if(b>=1&&p.trim(f.replace(bK,""))===""&&c.removeAttribute){c.removeAttribute("filter");if(d&&!d.filter)return}c.filter=bK.test(f)?f.replace(bK,e):f+" "+e}}),p(function(){p.support.reliableMarginRight||(p.cssHooks.marginRight={get:function(a,b){return p.swap(a,{display:"inline-block"},function(){if(b)return bH(a,"marginRight")})}}),!p.support.pixelPosition&&p.fn.position&&p.each(["top","left"
 ],function(a,b){p.cssHooks[b]={get:function(a,c){if(c){var d=bH(a,b);return bQ.test(d)?p(a).position()[b]+"px":d}}}})}),p.expr&&p.expr.filters&&(p.expr.filters.hidden=function(a){return a.offsetWidth===0&&a.offsetHeight===0||!p.support.reliableHiddenOffsets&&(a.style&&a.style.display||bH(a,"display"))==="none"},p.expr.filters.visible=function(a){return!p.expr.filters.hidden(a)}),p.each({margin:"",padding:"",border:"Width"},function(a,b){p.cssHooks[a+b]={expand:function(c){var d,e=typeof c=="string"?c.split(" "):[c],f={};for(d=0;d<4;d++)f[a+bV[d]+b]=e[d]||e[d-2]||e[0];return f}},bO.test(a)||(p.cssHooks[a+b].set=b_)});var cd=/%20/g,ce=/\[\]$/,cf=/\r?\n/g,cg=/^(?:color|date|datetime|datetime-local|email|hidden|month|number|password|range|search|tel|text|time|url|week)$/i,ch=/^(?:select|textarea)/i;p.fn.extend({serialize:function(){return p.param(this.serializeArray())},serializeArray:function(){return this.map(function(){return this.elements?p.makeArray(this.elements):this}).filter(fun
 ction(){return this.name&&!this.disabled&&(this.checked||ch.test(this.nodeName)||cg.test(this.type))}).map(function(a,b){var c=p(this).val();return c==null?null:p.isArray(c)?p.map(c,function(a,c){return{name:b.name,value:a.replace(cf,"\r\n")}}):{name:b.name,value:c.replace(cf,"\r\n")}}).get()}}),p.param=function(a,c){var d,e=[],f=function(a,b){b=p.isFunction(b)?b():b==null?"":b,e[e.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};c===b&&(c=p.ajaxSettings&&p.ajaxSettings.traditional);if(p.isArray(a)||a.jquery&&!p.isPlainObject(a))p.each(a,function(){f(this.name,this.value)});else for(d in a)ci(d,a[d],c,f);return e.join("&").replace(cd,"+")};var cj,ck,cl=/#.*$/,cm=/^(.*?):[ \t]*([^\r\n]*)\r?$/mg,cn=/^(?:about|app|app\-storage|.+\-extension|file|res|widget):$/,co=/^(?:GET|HEAD)$/,cp=/^\/\//,cq=/\?/,cr=/<script\b[^<]*(?:(?!<\/script>)<[^<]*)*<\/script>/gi,cs=/([?&])_=[^&]*/,ct=/^([\w\+\.\-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,cu=p.fn.load,cv={},cw={},cx=["*/"]+["*"];try{ck=f.href
 }catch(cy){ck=e.createElement("a"),ck.href="",ck=ck.href}cj=ct.exec(ck.toLowerCase())||[],p.fn.load=function(a,c,d){if(typeof a!="string"&&cu)return cu.apply(this,arguments);if(!this.length)return this;var e,f,g,h=this,i=a.indexOf(" ");return i>=0&&(e=a.slice(i,a.length),a=a.slice(0,i)),p.isFunction(c)?(d=c,c=b):c&&typeof c=="object"&&(f="POST"),p.ajax({url:a,type:f,dataType:"html",data:c,complete:function(a,b){d&&h.each(d,g||[a.responseText,b,a])}}).done(function(a){g=arguments,h.html(e?p("<div>").append(a.replace(cr,"")).find(e):a)}),this},p.each("ajaxStart ajaxStop ajaxComplete ajaxError ajaxSuccess ajaxSend".split(" "),function(a,b){p.fn[b]=function(a){return this.on(b,a)}}),p.each(["get","post"],function(a,c){p[c]=function(a,d,e,f){return p.isFunction(d)&&(f=f||e,e=d,d=b),p.ajax({type:c,url:a,data:d,success:e,dataType:f})}}),p.extend({getScript:function(a,c){return p.get(a,b,c,"script")},getJSON:function(a,b,c){return p.get(a,b,c,"json")},ajaxSetup:function(a,b){return b?cB(a,p
 .ajaxSettings):(b=a,a=p.ajaxSettings),cB(a,b),a},ajaxSettings:{url:ck,isLocal:cn.test(cj[1]),global:!0,type:"GET",contentType:"application/x-www-form-urlencoded; charset=UTF-8",processData:!0,async:!0,accepts:{xml:"application/xml, text/xml",html:"text/html",text:"text/plain",json:"application/json, text/javascript","*":cx},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":a.String,"text html":!0,"text json":p.parseJSON,"text xml":p.parseXML},flatOptions:{context:!0,url:!0}},ajaxPrefilter:cz(cv),ajaxTransport:cz(cw),ajax:function(a,c){function y(a,c,f,i){var k,s,t,u,w,y=c;if(v===2)return;v=2,h&&clearTimeout(h),g=b,e=i||"",x.readyState=a>0?4:0,f&&(u=cC(l,x,f));if(a>=200&&a<300||a===304)l.ifModified&&(w=x.getResponseHeader("Last-Modified"),w&&(p.lastModified[d]=w),w=x.getResponseHeader("Etag"),w&&(p.etag[d]=w)),a===304?(y="notmodified",k=!0):(k=cD(l,u),y=k.state,s=k.data,t=k.error,k=!t);else{t=y;if(!y||a)y="error",
 a<0&&(a=0)}x.status=a,x.statusText=(c||y)+"",k?o.resolveWith(m,[s,y,x]):o.rejectWith(m,[x,y,t]),x.statusCode(r),r=b,j&&n.trigger("ajax"+(k?"Success":"Error"),[x,l,k?s:t]),q.fireWith(m,[x,y]),j&&(n.trigger("ajaxComplete",[x,l]),--p.active||p.event.trigger("ajaxStop"))}typeof a=="object"&&(c=a,a=b),c=c||{};var d,e,f,g,h,i,j,k,l=p.ajaxSetup({},c),m=l.context||l,n=m!==l&&(m.nodeType||m instanceof p)?p(m):p.event,o=p.Deferred(),q=p.Callbacks("once memory"),r=l.statusCode||{},t={},u={},v=0,w="canceled",x={readyState:0,setRequestHeader:function(a,b){if(!v){var c=a.toLowerCase();a=u[c]=u[c]||a,t[a]=b}return this},getAllResponseHeaders:function(){return v===2?e:null},getResponseHeader:function(a){var c;if(v===2){if(!f){f={};while(c=cm.exec(e))f[c[1].toLowerCase()]=c[2]}c=f[a.toLowerCase()]}return c===b?null:c},overrideMimeType:function(a){return v||(l.mimeType=a),this},abort:function(a){return a=a||w,g&&g.abort(a),y(0,a),this}};o.promise(x),x.success=x.done,x.error=x.fail,x.complete=q.add,x.
 statusCode=function(a){if(a){var b;if(v<2)for(b in a)r[b]=[r[b],a[b]];else b=a[x.status],x.always(b)}return this},l.url=((a||l.url)+"").replace(cl,"").replace(cp,cj[1]+"//"),l.dataTypes=p.trim(l.dataType||"*").toLowerCase().split(s),l.crossDomain==null&&(i=ct.exec(l.url.toLowerCase())||!1,l.crossDomain=i&&i.join(":")+(i[3]?"":i[1]==="http:"?80:443)!==cj.join(":")+(cj[3]?"":cj[1]==="http:"?80:443)),l.data&&l.processData&&typeof l.data!="string"&&(l.data=p.param(l.data,l.traditional)),cA(cv,l,c,x);if(v===2)return x;j=l.global,l.type=l.type.toUpperCase(),l.hasContent=!co.test(l.type),j&&p.active++===0&&p.event.trigger("ajaxStart");if(!l.hasContent){l.data&&(l.url+=(cq.test(l.url)?"&":"?")+l.data,delete l.data),d=l.url;if(l.cache===!1){var z=p.now(),A=l.url.replace(cs,"$1_="+z);l.url=A+(A===l.url?(cq.test(l.url)?"&":"?")+"_="+z:"")}}(l.data&&l.hasContent&&l.contentType!==!1||c.contentType)&&x.setRequestHeader("Content-Type",l.contentType),l.ifModified&&(d=d||l.url,p.lastModified[d]&&x.s
 etRequestHeader("If-Modified-Since",p.lastModified[d]),p.etag[d]&&x.setRequestHeader("If-None-Match",p.etag[d])),x.setRequestHeader("Accept",l.dataTypes[0]&&l.accepts[l.dataTypes[0]]?l.accepts[l.dataTypes[0]]+(l.dataTypes[0]!=="*"?", "+cx+"; q=0.01":""):l.accepts["*"]);for(k in l.headers)x.setRequestHeader(k,l.headers[k]);if(!l.beforeSend||l.beforeSend.call(m,x,l)!==!1&&v!==2){w="abort";for(k in{success:1,error:1,complete:1})x[k](l[k]);g=cA(cw,l,c,x);if(!g)y(-1,"No Transport");else{x.readyState=1,j&&n.trigger("ajaxSend",[x,l]),l.async&&l.timeout>0&&(h=setTimeout(function(){x.abort("timeout")},l.timeout));try{v=1,g.send(t,y)}catch(B){if(v<2)y(-1,B);else throw B}}return x}return x.abort()},active:0,lastModified:{},etag:{}});var cE=[],cF=/\?/,cG=/(=)\?(?=&|$)|\?\?/,cH=p.now();p.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var a=cE.pop()||p.expando+"_"+cH++;return this[a]=!0,a}}),p.ajaxPrefilter("json jsonp",function(c,d,e){var f,g,h,i=c.data,j=c.url,k=c.jsonp!==!1,l=k&&cG.test(
 j),m=k&&!l&&typeof i=="string"&&!(c.contentType||"").indexOf("application/x-www-form-urlencoded")&&cG.test(i);if(c.dataTypes[0]==="jsonp"||l||m)return f=c.jsonpCallback=p.isFunction(c.jsonpCallback)?c.jsonpCallback():c.jsonpCallback,g=a[f],l?c.url=j.replace(cG,"$1"+f):m?c.data=i.replace(cG,"$1"+f):k&&(c.url+=(cF.test(j)?"&":"?")+c.jsonp+"="+f),c.converters["script json"]=function(){return h||p.error(f+" was not called"),h[0]},c.dataTypes[0]="json",a[f]=function(){h=arguments},e.always(function(){a[f]=g,c[f]&&(c.jsonpCallback=d.jsonpCallback,cE.push(f)),h&&p.isFunction(g)&&g(h[0]),h=g=b}),"script"}),p.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/javascript|ecmascript/},converters:{"text script":function(a){return p.globalEval(a),a}}}),p.ajaxPrefilter("script",function(a){a.cache===b&&(a.cache=!1),a.crossDomain&&(a.type="GET",a.global=!1)}),p.ajaxTransport("script",function(a){if(a.crossDomain
 ){var c,d=e.head||e.getElementsByTagName("head")[0]||e.documentElement;return{send:function(f,g){c=e.createElement("script"),c.async="async",a.scriptCharset&&(c.charset=a.scriptCharset),c.src=a.url,c.onload=c.onreadystatechange=function(a,e){if(e||!c.readyState||/loaded|complete/.test(c.readyState))c.onload=c.onreadystatechange=null,d&&c.parentNode&&d.removeChild(c),c=b,e||g(200,"success")},d.insertBefore(c,d.firstChild)},abort:function(){c&&c.onload(0,1)}}}});var cI,cJ=a.ActiveXObject?function(){for(var a in cI)cI[a](0,1)}:!1,cK=0;p.ajaxSettings.xhr=a.ActiveXObject?function(){return!this.isLocal&&cL()||cM()}:cL,function(a){p.extend(p.support,{ajax:!!a,cors:!!a&&"withCredentials"in a})}(p.ajaxSettings.xhr()),p.support.ajax&&p.ajaxTransport(function(c){if(!c.crossDomain||p.support.cors){var d;return{send:function(e,f){var g,h,i=c.xhr();c.username?i.open(c.type,c.url,c.async,c.username,c.password):i.open(c.type,c.url,c.async);if(c.xhrFields)for(h in c.xhrFields)i[h]=c.xhrFields[h];c.m
 imeType&&i.overrideMimeType&&i.overrideMimeType(c.mimeType),!c.crossDomain&&!e["X-Requested-With"]&&(e["X-Requested-With"]="XMLHttpRequest");try{for(h in e)i.setRequestHeader(h,e[h])}catch(j){}i.send(c.hasContent&&c.data||null),d=function(a,e){var h,j,k,l,m;try{if(d&&(e||i.readyState===4)){d=b,g&&(i.onreadystatechange=p.noop,cJ&&delete cI[g]);if(e)i.readyState!==4&&i.abort();else{h=i.status,k=i.getAllResponseHeaders(),l={},m=i.responseXML,m&&m.documentElement&&(l.xml=m);try{l.text=i.responseText}catch(a){}try{j=i.statusText}catch(n){j=""}!h&&c.isLocal&&!c.crossDomain?h=l.text?200:404:h===1223&&(h=204)}}}catch(o){e||f(-1,o)}l&&f(h,j,l,k)},c.async?i.readyState===4?setTimeout(d,0):(g=++cK,cJ&&(cI||(cI={},p(a).unload(cJ)),cI[g]=d),i.onreadystatechange=d):d()},abort:function(){d&&d(0,1)}}}});var cN,cO,cP=/^(?:toggle|show|hide)$/,cQ=new RegExp("^(?:([-+])=|)("+q+")([a-z%]*)$","i"),cR=/queueHooks$/,cS=[cY],cT={"*":[function(a,b){var c,d,e=this.createTween(a,b),f=cQ.exec(b),g=e.cur(),h=+g||
 0,i=1,j=20;if(f){c=+f[2],d=f[3]||(p.cssNumber[a]?"":"px");if(d!=="px"&&h){h=p.css(e.elem,a,!0)||c||1;do i=i||".5",h=h/i,p.style(e.elem,a,h+d);while(i!==(i=e.cur()/g)&&i!==1&&--j)}e.unit=d,e.start=h,e.end=f[1]?h+(f[1]+1)*c:c}return e}]};p.Animation=p.extend(cW,{tweener:function(a,b){p.isFunction(a)?(b=a,a=["*"]):a=a.split(" ");var c,d=0,e=a.length;for(;d<e;d++)c=a[d],cT[c]=cT[c]||[],cT[c].unshift(b)},prefilter:function(a,b){b?cS.unshift(a):cS.push(a)}}),p.Tween=cZ,cZ.prototype={constructor:cZ,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||"swing",this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(p.cssNumber[c]?"":"px")},cur:function(){var a=cZ.propHooks[this.prop];return a&&a.get?a.get(this):cZ.propHooks._default.get(this)},run:function(a){var b,c=cZ.propHooks[this.prop];return this.options.duration?this.pos=b=p.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):this.pos=b=a,this.now=(this.end-this.start)*b+this.start,this
 .options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):cZ.propHooks._default.set(this),this}},cZ.prototype.init.prototype=cZ.prototype,cZ.propHooks={_default:{get:function(a){var b;return a.elem[a.prop]==null||!!a.elem.style&&a.elem.style[a.prop]!=null?(b=p.css(a.elem,a.prop,!1,""),!b||b==="auto"?0:b):a.elem[a.prop]},set:function(a){p.fx.step[a.prop]?p.fx.step[a.prop](a):a.elem.style&&(a.elem.style[p.cssProps[a.prop]]!=null||p.cssHooks[a.prop])?p.style(a.elem,a.prop,a.now+a.unit):a.elem[a.prop]=a.now}}},cZ.propHooks.scrollTop=cZ.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},p.each(["toggle","show","hide"],function(a,b){var c=p.fn[b];p.fn[b]=function(d,e,f){return d==null||typeof d=="boolean"||!a&&p.isFunction(d)&&p.isFunction(e)?c.apply(this,arguments):this.animate(c$(b,!0),d,e,f)}}),p.fn.extend({fadeTo:function(a,b,c,d){return this.filter(bZ).css("opacity",0).show().end().animate({opacity:b},a,c,d)},a
 nimate:function(a,b,c,d){var 

<TRUNCATED>

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


[36/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 c452fd9..e77cfb1 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
@@ -32,6 +32,7 @@ import java.util.Map;
 import javax.ws.rs.core.MediaType;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -130,7 +131,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs").queryParam("state", notInUse.toString())
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("jobs is not empty",
@@ -152,7 +154,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("state", queryState)
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -172,7 +175,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -197,7 +201,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("user", "bogus")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("jobs is not empty",
@@ -210,7 +215,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("user", "mock")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     System.out.println(json.toString());
 
@@ -230,7 +236,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("limit", "2")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -248,7 +255,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -269,7 +277,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("queue", "mockqueue")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -283,7 +292,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("queue", "bogus")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("jobs is not empty",
@@ -299,7 +309,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs")
         .queryParam("startedTimeEnd", String.valueOf(now))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -316,7 +327,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs")
         .queryParam("startedTimeBegin", String.valueOf(now))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("jobs is not empty",
@@ -343,7 +355,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("startedTimeBegin", String.valueOf(40000))
         .queryParam("startedTimeEnd", String.valueOf(midStartTime))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -362,7 +375,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("startedTimeEnd", String.valueOf(40000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -388,7 +402,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs").queryParam("startedTimeBegin", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -414,7 +429,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs").queryParam("startedTimeEnd", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -440,7 +456,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("startedTimeBegin", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -466,7 +483,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("startedTimeEnd", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -490,7 +508,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("finishedTimeEnd", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -514,7 +533,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("finishedTimeBegin", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -541,7 +561,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("finishedTimeEnd", String.valueOf(40000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -567,7 +588,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs").queryParam("finishedTimeBegin", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -593,7 +615,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs").queryParam("finishedTimeEnd", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
     assertEquals("incorrect number of elements", 3, exception.length());
@@ -620,7 +643,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs")
         .queryParam("finishedTimeBegin", String.valueOf(now))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
@@ -637,7 +661,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .path("mapreduce").path("jobs")
         .queryParam("finishedTimeEnd", String.valueOf(now))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     assertEquals("jobs is not empty",
@@ -665,7 +690,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTestBase {
         .queryParam("finishedTimeBegin", String.valueOf(40000))
         .queryParam("finishedTimeEnd", String.valueOf(midFinishTime))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 0dbf29e..867ed61 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
@@ -34,6 +34,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
@@ -131,7 +132,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("tasks")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -151,7 +153,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("tasks")
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -171,7 +174,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("tasks/")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -192,7 +196,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       ClientResponse response = r.path("ws").path("v1").path("history")
           .path("mapreduce").path("jobs").path(jobId).path("tasks")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       String xml = response.getEntity(String.class);
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
       DocumentBuilder db = dbf.newDocumentBuilder();
@@ -217,7 +222,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
           .path("mapreduce").path("jobs").path(jobId).path("tasks")
           .queryParam("type", type).accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -238,7 +244,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
           .path("mapreduce").path("jobs").path(jobId).path("tasks")
           .queryParam("type", type).accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+          response.getType().toString());
       JSONObject json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
       JSONObject tasks = json.getJSONObject("tasks");
@@ -265,7 +272,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -294,7 +302,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("history")
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("task");
@@ -316,7 +325,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks")
             .path(tid + "/").accept(MediaType.APPLICATION_JSON)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("task");
@@ -337,7 +347,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("history")
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("task");
@@ -360,7 +371,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -394,7 +406,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -426,7 +439,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -460,7 +474,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -494,7 +509,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
         assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
         assertEquals("incorrect number of elements", 3, exception.length());
@@ -527,7 +543,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
 
-        assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         String xml = response.getEntity(String.class);
         DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
         DocumentBuilder db = dbf.newDocumentBuilder();
@@ -638,7 +655,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("counters").accept(MediaType.APPLICATION_JSON)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("jobTaskCounters");
@@ -660,7 +678,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("counters/").accept(MediaType.APPLICATION_JSON)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("jobTaskCounters");
@@ -681,7 +700,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
         ClientResponse response = r.path("ws").path("v1").path("history")
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("counters").get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         JSONObject json = response.getEntity(JSONObject.class);
         assertEquals("incorrect number of elements", 1, json.length());
         JSONObject info = json.getJSONObject("jobTaskCounters");
@@ -703,7 +723,8 @@ public class TestHsWebServicesTasks extends JerseyTestBase {
             .path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
             .path("counters").accept(MediaType.APPLICATION_XML)
             .get(ClientResponse.class);
-        assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+        assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+            response.getType().toString());
         String xml = response.getEntity(String.class);
         DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
         DocumentBuilder db = dbf.newDocumentBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index 1f657cf..5df1af5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.mapred;
 
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
@@ -76,7 +77,8 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     }
     webServer = new Server(0);
 
-    Context context = new Context(webServer, contextPath);
+    ServletContextHandler context =
+        new ServletContextHandler(webServer, contextPath);
 
     // create servlet handler
     context.addServlet(new ServletHolder(new NotificationServlet()),
@@ -84,7 +86,7 @@ public abstract class NotificationTestCase extends HadoopTestCase {
 
     // Start webServer
     webServer.start();
-    port = webServer.getConnectors()[0].getLocalPort();
+    port = ((ServerConnector)webServer.getConnectors()[0]).getLocalPort();
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
index 2e8ba5e..3cc73b5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.mapred.MiniMRClientClusterFactory;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 
 /**
  * This class drives the creation of a mini-cluster on the local machine. By

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 4c18709..9547062 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
@@ -127,7 +127,7 @@ import org.jboss.netty.handler.codec.http.QueryStringDecoder;
 import org.jboss.netty.handler.ssl.SslHandler;
 import org.jboss.netty.handler.stream.ChunkedWriteHandler;
 import org.jboss.netty.util.CharsetUtil;
-import org.mortbay.jetty.HttpHeaders;
+import org.eclipse.jetty.http.HttpHeader;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
@@ -1137,13 +1137,15 @@ public class ShuffleHandler extends AuxiliaryService {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Setting connection close header...");
         }
-        response.headers().set(HttpHeaders.CONNECTION, CONNECTION_CLOSE);
+        response.headers().set(HttpHeader.CONNECTION.asString(),
+            CONNECTION_CLOSE);
       } else {
-        response.headers().set(HttpHeaders.CONTENT_LENGTH,
+        response.headers().set(HttpHeader.CONTENT_LENGTH.asString(),
           String.valueOf(contentLength));
-        response.headers().set(HttpHeaders.CONNECTION, HttpHeaders.KEEP_ALIVE);
-        response.headers().set(HttpHeaders.KEEP_ALIVE, "timeout="
-            + connectionKeepAliveTimeOut);
+        response.headers().set(HttpHeader.CONNECTION.asString(),
+            HttpHeader.KEEP_ALIVE.asString());
+        response.headers().set(HttpHeader.KEEP_ALIVE.asString(),
+            "timeout=" + connectionKeepAliveTimeOut);
         LOG.info("Content Length in shuffle : " + contentLength);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 a927bf4..1e43937 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
@@ -92,7 +92,7 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.Mockito;
-import org.mortbay.jetty.HttpHeaders;
+import org.eclipse.jetty.http.HttpHeader;
 
 public class TestShuffleHandler {
   static final long MiB = 1024 * 1024; 
@@ -299,7 +299,8 @@ public class TestShuffleHandler {
     conn.connect();
     DataInputStream input = new DataInputStream(conn.getInputStream());
     Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-    Assert.assertEquals("close", conn.getHeaderField(HttpHeaders.CONNECTION));
+    Assert.assertEquals("close",
+        conn.getHeaderField(HttpHeader.CONNECTION.asString()));
     ShuffleHeader header = new ShuffleHeader();
     header.readFields(input);
     input.close();
@@ -409,15 +410,15 @@ public class TestShuffleHandler {
             + "map=attempt_12345_1_m_1_0");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-      ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+        ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
     conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-      ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+        ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
     conn.connect();
     DataInputStream input = new DataInputStream(conn.getInputStream());
-    Assert.assertEquals(HttpHeaders.KEEP_ALIVE,
-      conn.getHeaderField(HttpHeaders.CONNECTION));
+    Assert.assertEquals(HttpHeader.KEEP_ALIVE.asString(),
+        conn.getHeaderField(HttpHeader.CONNECTION.asString()));
     Assert.assertEquals("timeout=1",
-      conn.getHeaderField(HttpHeaders.KEEP_ALIVE));
+        conn.getHeaderField(HttpHeader.KEEP_ALIVE.asString()));
     Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
     ShuffleHeader header = new ShuffleHeader();
     header.readFields(input);
@@ -429,15 +430,15 @@ public class TestShuffleHandler {
             + "map=attempt_12345_1_m_1_0&keepAlive=true");
     conn = (HttpURLConnection) url.openConnection();
     conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-      ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+        ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
     conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-      ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+        ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
     conn.connect();
     input = new DataInputStream(conn.getInputStream());
-    Assert.assertEquals(HttpHeaders.KEEP_ALIVE,
-      conn.getHeaderField(HttpHeaders.CONNECTION));
+    Assert.assertEquals(HttpHeader.KEEP_ALIVE.asString(),
+        conn.getHeaderField(HttpHeader.CONNECTION.asString()));
     Assert.assertEquals("timeout=1",
-      conn.getHeaderField(HttpHeaders.KEEP_ALIVE));
+        conn.getHeaderField(HttpHeader.KEEP_ALIVE.asString()));
     Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
     header = new ShuffleHeader();
     header.readFields(input);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index e71d21f..b5041bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -46,8 +46,8 @@
       <artifactId>avro</artifactId>
       <exclusions>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
           <groupId>org.apache.ant</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-mapreduce-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/pom.xml b/hadoop-mapreduce-project/pom.xml
index 5fd346d..a1b437e 100644
--- a/hadoop-mapreduce-project/pom.xml
+++ b/hadoop-mapreduce-project/pom.xml
@@ -52,8 +52,8 @@
       <artifactId>avro</artifactId>
       <exclusions>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
         </exclusion>
         <exclusion>
           <groupId>org.apache.ant</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml
index 48ee5d5..f82d750 100644
--- a/hadoop-maven-plugins/pom.xml
+++ b/hadoop-maven-plugins/pom.xml
@@ -55,6 +55,10 @@
       <groupId>org.codehaus.jackson</groupId>
       <artifactId>jackson-mapper-asl</artifactId>
     </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
   </dependencies>
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/resourcegz/ResourceGzMojo.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/resourcegz/ResourceGzMojo.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/resourcegz/ResourceGzMojo.java
new file mode 100644
index 0000000..5c9e26e
--- /dev/null
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/resourcegz/ResourceGzMojo.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.maven.plugin.resourcegz;
+
+import com.google.inject.internal.util.Lists;
+import org.apache.commons.io.IOUtils;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.MojoFailureException;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.zip.GZIPOutputStream;
+
+/**
+ * ResourceGzMojo will gzip files.
+ * It is meant to be used for gzipping website resource files (e.g. .js, .css,
+ * etc).  It takes an input directory, output directory, and extensions to
+ * process and will generate the .gz files. Any additional directory structure
+ * beyond the input directory is preserved in the output directory.
+ */
+@Mojo(name="resource-gz")
+public class ResourceGzMojo extends AbstractMojo {
+
+  /**
+   * The input directory.  Will be searched recursively and its directory
+   * structure will be maintaned in the outputDirectory.
+   */
+  @Parameter(property = "inputDirectory", required = true)
+  private String inputDirectory;
+
+  /**
+   * The output directory.
+   */
+  @Parameter(property = "outputDirectory", required = true)
+  private String outputDirectory;
+
+  /**
+   * A comma separated list of extensions to include.
+   */
+  @Parameter(property = "extensions", required = true)
+  private String extensions;
+
+  public void execute() throws MojoExecutionException, MojoFailureException {
+    try {
+      Path inputDir = new File(inputDirectory).toPath();
+      File outputDir = new File(outputDirectory);
+      List<String> exts = Lists.newArrayList(extensions.split(","));
+      exts.replaceAll(String::trim);
+      GZConsumer cons = new GZConsumer(inputDir.toFile(), outputDir);
+      Files.walk(inputDir).filter(path -> {
+        for (String ext : exts) {
+          if (path.getFileName().toString().endsWith("." + ext)) {
+            return true;
+          }
+        }
+        return false;
+      }).forEach(cons);
+      if (cons.getThrowable() != null) {
+        throw new MojoExecutionException(cons.getThrowable().toString(),
+            cons.getThrowable());
+      }
+    } catch (Throwable t) {
+      throw new MojoExecutionException(t.toString(), t);
+    }
+  }
+
+  private class GZConsumer implements Consumer<Path> {
+    private final File inputDir;
+    private final File outputDir;
+    private Throwable throwable;
+
+    public GZConsumer(File inputDir, File outputDir) {
+      this.inputDir = inputDir;
+      this.outputDir = outputDir;
+      this.throwable = null;
+    }
+
+    @Override
+    public void accept(Path path) {
+      if (throwable != null) {
+        return;
+      }
+      try {
+        File outFile = new File(outputDir, path.toFile().getCanonicalPath()
+            .replaceFirst(inputDir.getCanonicalPath(), "") + ".gz");
+        outFile.getParentFile().mkdirs();
+        try (
+            GZIPOutputStream os = new GZIPOutputStream(
+                new FileOutputStream(outFile));
+            BufferedReader is = Files.newBufferedReader(path)
+        ) {
+          getLog().info("Compressing " + path + " to " + outFile);
+          IOUtils.copy(is, os);
+        } catch (Throwable t) {
+          this.throwable = t;
+        }
+      } catch (Throwable t) {
+        this.throwable = t;
+      }
+    }
+
+    public Throwable getThrowable() {
+      return throwable;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index f914f92..7e3b1cc 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -35,7 +35,7 @@
 
     <failIfNoTests>false</failIfNoTests>
     <maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
-    <jetty.version>6.1.26</jetty.version>
+    <jetty.version>9.3.11.v20160721</jetty.version>
     <test.exclude>_</test.exclude>
     <test.exclude.pattern>_</test.exclude.pattern>
 
@@ -518,28 +518,38 @@
       </dependency>
       <dependency>
         <groupId>javax.servlet</groupId>
-        <artifactId>servlet-api</artifactId>
-        <version>2.5</version>
+        <artifactId>javax.servlet-api</artifactId>
+        <version>3.1.0</version>
       </dependency>
       <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jetty</artifactId>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-server</artifactId>
         <version>${jetty.version}</version>
         <exclusions>
           <exclusion>
-            <groupId>org.mortbay.jetty</groupId>
-            <artifactId>servlet-api</artifactId>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>javax.servlet-api</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>org.mortbay.jetty</groupId>
+        <groupId>org.eclipse.jetty</groupId>
         <artifactId>jetty-util</artifactId>
         <version>${jetty.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jetty-sslengine</artifactId>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-servlet</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-webapp</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+        <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-util-ajax</artifactId>
         <version>${jetty.version}</version>
       </dependency>
       <dependency>
@@ -636,11 +646,23 @@
         <artifactId>jersey-test-framework-core</artifactId>
         <version>${jersey.version}</version>
         <scope>test</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>com.sun.jersey.jersey-test-framework</groupId>
         <artifactId>jersey-test-framework-grizzly2</artifactId>
         <version>${jersey.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
 
       <dependency>
@@ -668,7 +690,7 @@
       </dependency>
 
       <dependency>
-        <groupId>org.mortbay.jetty</groupId>
+        <groupId>org.eclipse.jetty</groupId>
         <artifactId>jetty-servlet-tester</artifactId>
         <version>${jetty.version}</version>
       </dependency>
@@ -687,7 +709,7 @@
           </exclusion>
           <exclusion>
             <groupId>javax.servlet</groupId>
-            <artifactId>servlet-api</artifactId>
+            <artifactId>javax.servlet-api</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
@@ -847,6 +869,12 @@
         <groupId>org.mock-server</groupId>
         <artifactId>mockserver-netty</artifactId>
         <version>3.9.2</version>
+        <exclusions>
+          <exclusion>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>org.apache.avro</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index e1a0bfe..9a15b04 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -135,7 +135,7 @@
       <artifactId>hadoop-client</artifactId>
       <exclusions>
         <exclusion>
-          <artifactId>servlet-api</artifactId>
+          <artifactId>javax.servlet-api</artifactId>
           <groupId>javax.servlet</groupId>
         </exclusion>
       </exclusions>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-tools/hadoop-azure/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index d8121e2..c4b45f4 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -164,13 +164,18 @@
       <artifactId>azure-storage</artifactId>
       <scope>compile</scope>
     </dependency>
-    
 
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
       <scope>compile</scope>
     </dependency>
+
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
+      <scope>compile</scope>
+    </dependency>
     
     
     <!-- dependencies use for test only -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index eaca82e..3e864a4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -59,7 +59,7 @@ import org.apache.hadoop.fs.azure.metrics.ResponseReceivedMetricUpdater;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.io.IOUtils;
-import org.mortbay.util.ajax.JSON;
+import org.eclipse.jetty.util.ajax.JSON;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
index e419a3b..a7e286c 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
@@ -40,7 +40,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.fs.azure.StorageInterface.CloudBlockBlobWrapper;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -760,7 +760,7 @@ public class BlockBlobAppendStream extends OutputStream {
               dataPayload.length, new BlobRequestOptions(), opContext);
           break;
         } catch(Exception ioe) {
-          Log.debug("Encountered exception during uploading block for Blob : {} Exception : {}", key, ioe);
+          Log.getLog().debug("Encountered exception during uploading block for Blob : {} Exception : {}", key, ioe);
           uploadRetryAttempts++;
           lastLocalException = new IOException("Encountered Exception while uploading block", ioe);
           try {
@@ -814,7 +814,7 @@ public class BlockBlobAppendStream extends OutputStream {
           Thread.currentThread().interrupt();
         }
 
-        Log.debug("Attempting to renew append lease on {}", key);
+        Log.getLog().debug("Attempting to renew append lease on {}", key);
 
         try {
           if (!leaseFreed) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-tools/hadoop-sls/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/pom.xml b/hadoop-tools/hadoop-sls/pom.xml
index da70b24..b84f5fc 100644
--- a/hadoop-tools/hadoop-sls/pom.xml
+++ b/hadoop-tools/hadoop-sls/pom.xml
@@ -55,18 +55,18 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
       <scope>provided</scope>
       <exclusions>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>servlet-api</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>javax.servlet-api</artifactId>
         </exclusion>
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
       <scope>provided</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
index e152696..abdf106 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
@@ -38,16 +39,16 @@ import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.scheduler.FairSchedulerMetrics;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
-import org.mortbay.jetty.Handler;
-import org.mortbay.jetty.Request;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.handler.AbstractHandler;
-import org.mortbay.jetty.handler.ResourceHandler;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.Server;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.MetricRegistry;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+import org.eclipse.jetty.server.handler.ResourceHandler;
 
 @Private
 @Unstable
@@ -121,8 +122,10 @@ public class SLSWebApp extends HttpServlet {
 
     Handler handler = new AbstractHandler() {
       @Override
-      public void handle(String target, HttpServletRequest request,
-                         HttpServletResponse response, int dispatch) {
+      public void handle(String target, Request baseRequest,
+                         HttpServletRequest request,
+                         HttpServletResponse response)
+          throws IOException, ServletException {
         try{
           // timeunit
           int timeunit = 1000;   // second, divide millionsecond / 1000
@@ -144,7 +147,7 @@ public class SLSWebApp extends HttpServlet {
             // js/css request
             if (target.startsWith("/js") || target.startsWith("/css")) {
               response.setCharacterEncoding("utf-8");
-              staticHandler.handle(target, request, response, dispatch);
+              staticHandler.handle(target, baseRequest, request, response);
             } else
               // json request
               if (target.equals("/simulateMetrics")) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index c4bb3ce..8772fdc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -99,7 +99,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 import com.google.common.base.Supplier;
 
@@ -654,7 +654,8 @@ public class TestAMRMClient {
       int iterationsLeft = 3;
       while (allocatedContainerCount < 2
           && iterationsLeft-- > 0) {
-        Log.info(" == alloc " + allocatedContainerCount + " it left " + iterationsLeft);
+        Log.getLog().info("Allocated " + allocatedContainerCount + " containers"
+            + " with " + iterationsLeft + " iterations left");
         AllocateResponse allocResponse = amClient.allocate(0.1f);
         assertEquals(0, amClient.ask.size());
         assertEquals(0, amClient.release.size());
@@ -822,7 +823,8 @@ public class TestAMRMClient {
       throws YarnException, IOException {
     int allocatedContainerCount = 0;
     while (iterationsLeft-- > 0) {
-      Log.info(" == alloc " + allocatedContainerCount + " it left " + iterationsLeft);
+      Log.getLog().info("Allocated " + allocatedContainerCount + " containers"
+          + " with " + iterationsLeft + " iterations left");
       AllocateResponse allocResponse = amClient.allocate(0.1f);
       assertEquals(0, amClient.ask.size());
       assertEquals(0, amClient.release.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index c4d2e09..c1cabb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -84,7 +84,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.C
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.log.Log;
+import org.eclipse.jetty.util.log.Log;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
@@ -325,11 +325,11 @@ public class TestYarnCLI {
         "http://host:2345", "");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
-    Log.info("ExpectedOutput");
-    Log.info("["+appReportStr+"]");
-    Log.info("OutputFrom command");
+    Log.getLog().info("ExpectedOutput");
+    Log.getLog().info("["+appReportStr+"]");
+    Log.getLog().info("OutputFrom command");
     String actualOutput = sysOutStream.toString("UTF-8");
-    Log.info("["+actualOutput+"]");
+    Log.getLog().info("["+actualOutput+"]");
     Assert.assertEquals(appReportStr, actualOutput);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/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 8f31874..f1be098 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -68,14 +68,14 @@
     </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
     <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-util</artifactId>
     </dependency>
     <dependency>
@@ -279,7 +279,18 @@
               <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
-
+          <execution>
+            <id>resource-gz</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>resource-gz</goal>
+            </goals>
+            <configuration>
+              <inputDirectory>${basedir}/src/main/resources/webapps/static</inputDirectory>
+              <outputDirectory>${basedir}/target/classes/webapps/static</outputDirectory>
+              <extensions>js,css</extensions>
+            </configuration>
+          </execution>
         </executions>
       </plugin>
       <plugin>


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


[20/50] [abbrv] hadoop git commit: YARN-4765 Split TestHBaseTimelineStorage into multiple test classes (Varun Saxena via Vrushali C)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/022bf783/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
new file mode 100644
index 0000000..3076709
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
@@ -0,0 +1,1675 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Various tests to test writing entities to HBase and reading them back from
+ * it.
+ *
+ * It uses a single HBase mini-cluster for all tests which is a little more
+ * realistic, and helps test correctness in the presence of other data.
+ *
+ * Each test uses a different cluster name to be able to handle its own data
+ * even if other records exist in the table. Use a different cluster name if
+ * you add a new test.
+ */
+public class TestHBaseTimelineStorageEntities {
+
+  private static HBaseTestingUtility util;
+  private HBaseTimelineReaderImpl reader;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    util = new HBaseTestingUtility();
+    util.startMiniCluster();
+    createSchema();
+    DataGeneratorForTest.loadEntities(util);
+  }
+
+  private static void createSchema() throws IOException {
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
+  }
+
+  @Before
+  public void init() throws Exception {
+    reader = new HBaseTimelineReaderImpl();
+    reader.init(util.getConfiguration());
+    reader.start();
+  }
+
+  @After
+  public void stop() throws Exception {
+    if (reader != null) {
+      reader.stop();
+      reader.close();
+    }
+  }
+
+  private static void matchMetrics(Map<Long, Number> m1, Map<Long, Number> m2) {
+    assertEquals(m1.size(), m2.size());
+    for (Map.Entry<Long, Number> entry : m2.entrySet()) {
+      Number val = m1.get(entry.getKey());
+      assertNotNull(val);
+      assertEquals(val.longValue(), entry.getValue().longValue());
+    }
+  }
+
+  @Test
+  public void testWriteEntityToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "hello";
+    String type = "world";
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    long ts = System.currentTimeMillis();
+    metricValues.put(ts - 120000, 100000000);
+    metricValues.put(ts - 100000, 200000000);
+    metricValues.put(ts - 80000, 300000000);
+    metricValues.put(ts - 60000, 400000000);
+    metricValues.put(ts - 40000, 50000000000L);
+    metricValues.put(ts - 20000, 60000000000L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+    te.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_write_entity";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
+          9000000L, 1).toString();
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.stop();
+
+      // scan the table and see that entity exists
+      Scan s = new Scan();
+      byte[] startRow =
+          new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
+              .getRowKeyPrefix();
+      s.setStartRow(startRow);
+      s.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
+
+      int rowCount = 0;
+      int colCount = 0;
+      KeyConverter<String> stringKeyConverter = new StringKeyConverter();
+      for (Result result : scanner) {
+        if (result != null && !result.isEmpty()) {
+          rowCount++;
+          colCount += result.size();
+          byte[] row1 = result.getRow();
+          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
+              entity));
+
+          // check info column family
+          String id1 = EntityColumn.ID.readResult(result).toString();
+          assertEquals(id, id1);
+
+          String type1 = EntityColumn.TYPE.readResult(result).toString();
+          assertEquals(type, type1);
+
+          Long cTime1 = (Long) EntityColumn.CREATED_TIME.readResult(result);
+          assertEquals(cTime1, cTime);
+
+          Map<String, Object> infoColumns =
+              EntityColumnPrefix.INFO.readResults(result,
+                  new StringKeyConverter());
+          assertEquals(infoMap, infoColumns);
+
+          // Remember isRelatedTo is of type Map<String, Set<String>>
+          for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
+              .entrySet()) {
+            Object isRelatedToValue =
+                EntityColumnPrefix.IS_RELATED_TO.readResult(result,
+                    isRelatedToEntry.getKey());
+            String compoundValue = isRelatedToValue.toString();
+            // id7?id9?id6
+            Set<String> isRelatedToValues =
+                new HashSet<String>(
+                    Separator.VALUES.splitEncoded(compoundValue));
+            assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
+                isRelatedToValues.size());
+            for (String v : isRelatedToEntry.getValue()) {
+              assertTrue(isRelatedToValues.contains(v));
+            }
+          }
+
+          // RelatesTo
+          for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
+              .entrySet()) {
+            String compoundValue = EntityColumnPrefix.RELATES_TO
+                .readResult(result, relatesToEntry.getKey()).toString();
+            // id3?id4?id5
+            Set<String> relatesToValues =
+                new HashSet<String>(
+                    Separator.VALUES.splitEncoded(compoundValue));
+            assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
+                relatesToValues.size());
+            for (String v : relatesToEntry.getValue()) {
+              assertTrue(relatesToValues.contains(v));
+            }
+          }
+
+          // Configuration
+          Map<String, Object> configColumns =
+              EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter);
+          assertEquals(conf, configColumns);
+
+          NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result,
+                  stringKeyConverter);
+
+          NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
+          matchMetrics(metricValues, metricMap);
+        }
+      }
+      assertEquals(1, rowCount);
+      assertEquals(16, colCount);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
+          Integer.MAX_VALUE));
+      Set<TimelineEntity> es1 = reader.getEntities(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
+          Integer.MAX_VALUE));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+
+      // verify attributes
+      assertEquals(id, e1.getId());
+      assertEquals(type, e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      Map<String, Object> infoMap2 = e1.getInfo();
+      assertEquals(infoMap, infoMap2);
+
+      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
+      assertEquals(isRelatedTo, isRelatedTo2);
+
+      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
+      assertEquals(relatesTo, relatesTo2);
+
+      Map<String, String> conf2 = e1.getConfigs();
+      assertEquals(conf, conf2);
+
+      Set<TimelineMetric> metrics2 = e1.getMetrics();
+      assertEquals(metrics, metrics2);
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        matchMetrics(metricValues, metricValues2);
+      }
+
+      e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
+          runid, appName, entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      assertNotNull(e1);
+      assertEquals(id, e1.getId());
+      assertEquals(type, e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(infoMap, e1.getInfo());
+      assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
+      assertEquals(relatesTo, e1.getRelatesToEntities());
+      assertEquals(conf, e1.getConfigs());
+      for (TimelineMetric metric : e1.getMetrics()) {
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertEquals(1, metric.getValues().size());
+        assertTrue(metric.getValues().containsKey(ts - 20000));
+        assertEquals(metricValues.get(ts - 20000),
+            metric.getValues().get(ts - 20000));
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
+      String flow, Long runid, String appName, TimelineEntity te) {
+
+    EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
+
+    assertEquals(user, key.getUserId());
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(flow, key.getFlowName());
+    assertEquals(runid, key.getFlowRunId());
+    assertEquals(appName, key.getAppId());
+    assertEquals(te.getType(), key.getEntityType());
+    assertEquals(te.getId(), key.getEntityId());
+    return true;
+  }
+
+  @Test
+  public void testEventsWithEmptyInfo() throws IOException {
+    TimelineEvent event = new TimelineEvent();
+    String eventId = "foo_ev e  nt_id";
+    event.setId(eventId);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+
+    final TimelineEntity entity = new TimelineEntity();
+    entity.setId("attempt_1329348432655_0001_m_000008_18");
+    entity.setType("FOO_ATTEMPT");
+    entity.addEvent(event);
+
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_empty_eventkey";
+      String user = "user_emptyeventkey";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
+          9000000L, 1).toString();
+      byte[] startRow =
+          new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
+              .getRowKeyPrefix();
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.stop();
+      // scan the table and see that entity exists
+      Scan s = new Scan();
+      s.setStartRow(startRow);
+      s.addFamily(EntityColumnFamily.INFO.getBytes());
+      Connection conn = ConnectionFactory.createConnection(c1);
+      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
+
+      int rowCount = 0;
+      for (Result result : scanner) {
+        if (result != null && !result.isEmpty()) {
+          rowCount++;
+
+          // check the row key
+          byte[] row1 = result.getRow();
+          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
+              entity));
+
+          Map<EventColumnName, Object> eventsResult =
+              EntityColumnPrefix.EVENT.readResults(result,
+                  new EventColumnNameConverter());
+          // there should be only one event
+          assertEquals(1, eventsResult.size());
+          for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
+            EventColumnName eventColumnName = e.getKey();
+            // the qualifier is a compound key
+            // hence match individual values
+            assertEquals(eventId, eventColumnName.getId());
+            assertEquals(expTs, eventColumnName.getTimestamp());
+            // key must be empty
+            assertNull(eventColumnName.getInfoKey());
+            Object value = e.getValue();
+            // value should be empty
+            assertEquals("", value.toString());
+          }
+        }
+      }
+      assertEquals(1, rowCount);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      Set<TimelineEntity> es1 = reader.getEntities(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, Long.valueOf(e.getTimestamp()));
+        Map<String, Object> info = e.getInfo();
+        assertTrue(info == null || info.isEmpty());
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Test
+  public void testEventsEscapeTs() throws IOException {
+    TimelineEvent event = new TimelineEvent();
+    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
+    event.setId(eventId);
+    long expTs = 1463567041056L;
+    event.setTimestamp(expTs);
+    String expKey = "f==o o_e ve\tnt";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+
+    final TimelineEntity entity = new ApplicationEntity();
+    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.addEvent(event);
+
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "clus!ter_\ttest_ev  ents";
+      String user = "user2";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = "application_123465899910_2001";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.stop();
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      assertNotNull(e1);
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, e.getTimestamp());
+        Map<String, Object> info = e.getInfo();
+        assertEquals(1, info.size());
+        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
+          assertEquals(expKey, infoEntry.getKey());
+          assertEquals(expVal, infoEntry.getValue());
+        }
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Test
+  public void testReadEntities() throws Exception {
+    TimelineEntity entity = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertNotNull(entity);
+    assertEquals(3, entity.getConfigs().size());
+    assertEquals(1, entity.getIsRelatedToEntities().size());
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world",
+        null), new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(3, entities.size());
+    int cfgCnt = 0;
+    int metricCnt = 0;
+    int infoCnt = 0;
+    int eventCnt = 0;
+    int relatesToCnt = 0;
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
+          timelineEntity.getConfigs().size();
+      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
+          timelineEntity.getMetrics().size();
+      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
+          timelineEntity.getInfo().size();
+      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
+          timelineEntity.getEvents().size();
+      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
+          timelineEntity.getRelatesToEntities().size();
+      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
+          timelineEntity.getIsRelatedToEntities().size();
+    }
+    assertEquals(5, cfgCnt);
+    assertEquals(3, metricCnt);
+    assertEquals(5, infoCnt);
+    assertEquals(4, eventCnt);
+    assertEquals(4, relatesToCnt);
+    assertEquals(4, isRelatedToCnt);
+  }
+
+  @Test
+  public void testFilterEntitiesByCreatedTime() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
+        null, null, null, null, null), new TimelineDataToRetrieve());
+    assertEquals(3, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
+          !entity.getId().equals("hello2")) {
+        Assert.fail("Entities with ids' hello, hello1 and hello2 should be" +
+            " present");
+      }
+    }
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
+        null, null, null), new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello1") &&
+          !entity.getId().equals("hello2")) {
+        Assert.fail("Entities with ids' hello1 and hello2 should be present");
+      }
+    }
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world",  null),
+        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
+        null, null, null), new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello")) {
+        Assert.fail("Entity with id hello should be present");
+      }
+    }
+  }
+
+  @Test
+  public void testReadEntitiesRelationsAndEventFiltersDefaultView()
+      throws Exception {
+    TimelineFilterList eventFilter = new TimelineFilterList();
+    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
+        "end_event"));
+    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList isRelatedTo = new TimelineFilterList();
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
+        null, null, null, eventFilter), new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    int isRelatedToCnt = 0;
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+    assertEquals(0, isRelatedToCnt);
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesEventFilters() throws Exception {
+    TimelineFilterList ef = new TimelineFilterList();
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(1, eventCnt);
+
+    TimelineFilterList ef1 = new TimelineFilterList();
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef1),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef2 = new TimelineFilterList();
+    ef2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef2),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef3 = new TimelineFilterList();
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef3),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "start_event"));
+    TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef4),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef5 = new TimelineFilterList();
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "update_event"));
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef5),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, eventCnt);
+  }
+
+  @Test
+  public void testReadEntitiesIsRelatedTo() throws Exception {
+    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity ids' should have been hello and hello1");
+      }
+    }
+    assertEquals(3, isRelatedToCnt);
+
+    TimelineFilterList irt1 = new TimelineFilterList();
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity ids' should have been hello and hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt3 = new TimelineFilterList();
+    irt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt4 = new TimelineFilterList();
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList irt5 = new TimelineFilterList();
+    irt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesRelatesTo() throws Exception {
+    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(3, relatesToCnt);
+
+    TimelineFilterList rt1 = new TimelineFilterList();
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt3 = new TimelineFilterList();
+    rt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt4 = new TimelineFilterList();
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList rt5 = new TimelineFilterList();
+    rt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList list3 = new TimelineFilterList();
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list4 = new TimelineFilterList();
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto2"))));
+    TimelineFilterList combinedList =
+        new TimelineFilterList(Operator.OR, list3, list4);
+    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
+        new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesDefaultView() throws Exception {
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
+        new TimelineDataToRetrieve());
+    assertNotNull(e1);
+    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
+        e1.getRelatesToEntities().isEmpty());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve());
+    assertEquals(3, es1.size());
+    for (TimelineEntity e : es1) {
+      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
+          e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
+          e.getRelatesToEntities().isEmpty());
+    }
+  }
+
+  @Test
+  public void testReadEntitiesByFields() throws Exception {
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
+        new TimelineDataToRetrieve(
+        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
+    assertNotNull(e1);
+    assertEquals(3, e1.getConfigs().size());
+    assertEquals(0, e1.getIsRelatedToEntities().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(
+        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
+    assertEquals(3, es1.size());
+    int metricsCnt = 0;
+    int isRelatedToCnt = 0;
+    int infoCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricsCnt += entity.getMetrics().size();
+      isRelatedToCnt += entity.getIsRelatedToEntities().size();
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(0, infoCnt);
+    assertEquals(4, isRelatedToCnt);
+    assertEquals(3, metricsCnt);
+  }
+
+  @Test
+  public void testReadEntitiesConfigPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
+        new TimelineDataToRetrieve(list, null, null, null));
+    assertNotNull(e1);
+    assertEquals(1, e1.getConfigs().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(list, null, null, null));
+    int cfgCnt = 0;
+    for (TimelineEntity entity : es1) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(3, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesConfigFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(2, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    TimelineFilterList confFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(1, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(3, cfgCnt);
+
+    TimelineFilterList confFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_param2", "value2"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList2, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList3, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+            new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+            1002345678919L, "application_1231111111_1111", "world", null),
+            new TimelineEntityFilters(null, null, null, null, null, null,
+            confFilterList4, null, null),
+            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+            null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList5, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadEntitiesConfigFilterPrefix() throws Exception {
+    TimelineFilterList confFilterList = new TimelineFilterList();
+    confFilterList.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(list, null, null, null));
+    assertEquals(1, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(2, cfgCnt);
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    TimelineFilterList confsToRetrieve =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with config_",
+            confKey.startsWith("config_"));
+      }
+    }
+    assertEquals(2, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesMetricPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
+        new TimelineDataToRetrieve(null, list, null, null));
+    assertNotNull(e1);
+    assertEquals(1, e1.getMetrics().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(null, list, null, null));
+    int metricCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+  }
+
+  @Test
+  public void testReadEntitiesMetricFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(2, entities.size());
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    TimelineFilterList metricFilterList1 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L),
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(1, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(2, metricCnt);
+
+    TimelineFilterList metricFilterList2 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 40000000000L),
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList2, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList3 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "dummy_metric", 5));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList3, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList4 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList4, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList5 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5, false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList5, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadEntitiesMetricFilterPrefix() throws Exception {
+    TimelineFilterList metricFilterList = new TimelineFilterList();
+    metricFilterList.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 0L));
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, list, null, null));
+    assertEquals(1, entities.size());
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(1, metricCnt);
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve(
+        null, metricsToRetrieve, EnumSet.of(Field.METRICS), null));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertEquals(1, metric.getValues().size());
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+
+    entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111",
+        "world", null), new TimelineEntityFilters(null, null, null, null, null,
+        null, null, metricFilterList1, null), new TimelineDataToRetrieve(null,
+        metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    int metricValCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        metricValCnt += metric.getValues().size();
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+    assertEquals(7, metricValCnt);
+  }
+
+  @Test
+  public void testReadEntitiesInfoFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey3", 71.4));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue1"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey2", 10));
+    TimelineFilterList infoFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(2, entities.size());
+    int infoCnt = 0;
+    for (TimelineEntity entity : entities) {
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(5, infoCnt);
+
+    TimelineFilterList infoFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(1, entities.size());
+    infoCnt = 0;
+    for (TimelineEntity entity : entities) {
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(3, infoCnt);
+
+    TimelineFilterList infoFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue2"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey3", 71.4));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_info", "some_value"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(3, entities.size());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}


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


[32/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js
new file mode 100644
index 0000000..aa7a923
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js
@@ -0,0 +1,6 @@
+/*! jQuery UI - v1.9.1 - 2012-10-25
+* http://jqueryui.com
+* Includes: jquery.ui.core.js, jquery.ui.widget.js, jquery.ui.mouse.js, jquery.ui.position.js, jquery.ui.accordion.js, jquery.ui.autocomplete.js, jquery.ui.button.js, jquery.ui.datepicker.js, jquery.ui.dialog.js, jquery.ui.draggable.js, jquery.ui.droppable.js, jquery.ui.effect.js, jquery.ui.effect-blind.js, jquery.ui.effect-bounce.js, jquery.ui.effect-clip.js, jquery.ui.effect-drop.js, jquery.ui.effect-explode.js, jquery.ui.effect-fade.js, jquery.ui.effect-fold.js, jquery.ui.effect-highlight.js, jquery.ui.effect-pulsate.js, jquery.ui.effect-scale.js, jquery.ui.effect-shake.js, jquery.ui.effect-slide.js, jquery.ui.effect-transfer.js, jquery.ui.menu.js, jquery.ui.progressbar.js, jquery.ui.resizable.js, jquery.ui.selectable.js, jquery.ui.slider.js, jquery.ui.sortable.js, jquery.ui.spinner.js, jquery.ui.tabs.js, jquery.ui.tooltip.js
+* Copyright (c) 2012 jQuery Foundation and other contributors Licensed MIT */
+
+(function(e,t){function i(t,n){var r,i,o,u=t.nodeName.toLowerCase();return"area"===u?(r=t.parentNode,i=r.name,!t.href||!i||r.nodeName.toLowerCase()!=="map"?!1:(o=e("img[usemap=#"+i+"]")[0],!!o&&s(o))):(/input|select|textarea|button|object/.test(u)?!t.disabled:"a"===u?t.href||n:n)&&s(t)}function s(t){return e.expr.filters.visible(t)&&!e(t).parents().andSelf().filter(function(){return e.css(this,"visibility")==="hidden"}).length}var n=0,r=/^ui-id-\d+$/;e.ui=e.ui||{};if(e.ui.version)return;e.extend(e.ui,{version:"1.9.1",keyCode:{BACKSPACE:8,COMMA:188,DELETE:46,DOWN:40,END:35,ENTER:13,ESCAPE:27,HOME:36,LEFT:37,NUMPAD_ADD:107,NUMPAD_DECIMAL:110,NUMPAD_DIVIDE:111,NUMPAD_ENTER:108,NUMPAD_MULTIPLY:106,NUMPAD_SUBTRACT:109,PAGE_DOWN:34,PAGE_UP:33,PERIOD:190,RIGHT:39,SPACE:32,TAB:9,UP:38}}),e.fn.extend({_focus:e.fn.focus,focus:function(t,n){return typeof t=="number"?this.each(function(){var r=this;setTimeout(function(){e(r).focus(),n&&n.call(r)},t)}):this._focus.apply(this,arguments)},scrollPa
 rent:function(){var t;return e.ui.ie&&/(static|relative)/.test(this.css("position"))||/absolute/.test(this.css("position"))?t=this.parents().filter(function(){return/(relative|absolute|fixed)/.test(e.css(this,"position"))&&/(auto|scroll)/.test(e.css(this,"overflow")+e.css(this,"overflow-y")+e.css(this,"overflow-x"))}).eq(0):t=this.parents().filter(function(){return/(auto|scroll)/.test(e.css(this,"overflow")+e.css(this,"overflow-y")+e.css(this,"overflow-x"))}).eq(0),/fixed/.test(this.css("position"))||!t.length?e(document):t},zIndex:function(n){if(n!==t)return this.css("zIndex",n);if(this.length){var r=e(this[0]),i,s;while(r.length&&r[0]!==document){i=r.css("position");if(i==="absolute"||i==="relative"||i==="fixed"){s=parseInt(r.css("zIndex"),10);if(!isNaN(s)&&s!==0)return s}r=r.parent()}}return 0},uniqueId:function(){return this.each(function(){this.id||(this.id="ui-id-"+ ++n)})},removeUniqueId:function(){return this.each(function(){r.test(this.id)&&e(this).removeAttr("id")})}}),e("
 <a>").outerWidth(1).jquery||e.each(["Width","Height"],function(n,r){function u(t,n,r,s){return e.each(i,function(){n-=parseFloat(e.css(t,"padding"+this))||0,r&&(n-=parseFloat(e.css(t,"border"+this+"Width"))||0),s&&(n-=parseFloat(e.css(t,"margin"+this))||0)}),n}var i=r==="Width"?["Left","Right"]:["Top","Bottom"],s=r.toLowerCase(),o={innerWidth:e.fn.innerWidth,innerHeight:e.fn.innerHeight,outerWidth:e.fn.outerWidth,outerHeight:e.fn.outerHeight};e.fn["inner"+r]=function(n){return n===t?o["inner"+r].call(this):this.each(function(){e(this).css(s,u(this,n)+"px")})},e.fn["outer"+r]=function(t,n){return typeof t!="number"?o["outer"+r].call(this,t):this.each(function(){e(this).css(s,u(this,t,!0,n)+"px")})}}),e.extend(e.expr[":"],{data:e.expr.createPseudo?e.expr.createPseudo(function(t){return function(n){return!!e.data(n,t)}}):function(t,n,r){return!!e.data(t,r[3])},focusable:function(t){return i(t,!isNaN(e.attr(t,"tabindex")))},tabbable:function(t){var n=e.attr(t,"tabindex"),r=isNaN(n);retu
 rn(r||n>=0)&&i(t,!r)}}),e(function(){var t=document.body,n=t.appendChild(n=document.createElement("div"));n.offsetHeight,e.extend(n.style,{minHeight:"100px",height:"auto",padding:0,borderWidth:0}),e.support.minHeight=n.offsetHeight===100,e.support.selectstart="onselectstart"in n,t.removeChild(n).style.display="none"}),function(){var t=/msie ([\w.]+)/.exec(navigator.userAgent.toLowerCase())||[];e.ui.ie=t.length?!0:!1,e.ui.ie6=parseFloat(t[1],10)===6}(),e.fn.extend({disableSelection:function(){return this.bind((e.support.selectstart?"selectstart":"mousedown")+".ui-disableSelection",function(e){e.preventDefault()})},enableSelection:function(){return this.unbind(".ui-disableSelection")}}),e.extend(e.ui,{plugin:{add:function(t,n,r){var i,s=e.ui[t].prototype;for(i in r)s.plugins[i]=s.plugins[i]||[],s.plugins[i].push([n,r[i]])},call:function(e,t,n){var r,i=e.plugins[t];if(!i||!e.element[0].parentNode||e.element[0].parentNode.nodeType===11)return;for(r=0;r<i.length;r++)e.options[i[r][0]]&&i
 [r][1].apply(e.element,n)}},contains:e.contains,hasScroll:function(t,n){if(e(t).css("overflow")==="hidden")return!1;var r=n&&n==="left"?"scrollLeft":"scrollTop",i=!1;return t[r]>0?!0:(t[r]=1,i=t[r]>0,t[r]=0,i)},isOverAxis:function(e,t,n){return e>t&&e<t+n},isOver:function(t,n,r,i,s,o){return e.ui.isOverAxis(t,r,s)&&e.ui.isOverAxis(n,i,o)}})})(jQuery);(function(e,t){var n=0,r=Array.prototype.slice,i=e.cleanData;e.cleanData=function(t){for(var n=0,r;(r=t[n])!=null;n++)try{e(r).triggerHandler("remove")}catch(s){}i(t)},e.widget=function(t,n,r){var i,s,o,u,a=t.split(".")[0];t=t.split(".")[1],i=a+"-"+t,r||(r=n,n=e.Widget),e.expr[":"][i.toLowerCase()]=function(t){return!!e.data(t,i)},e[a]=e[a]||{},s=e[a][t],o=e[a][t]=function(e,t){if(!this._createWidget)return new o(e,t);arguments.length&&this._createWidget(e,t)},e.extend(o,s,{version:r.version,_proto:e.extend({},r),_childConstructors:[]}),u=new n,u.options=e.widget.extend({},u.options),e.each(r,function(t,i){e.isFunction(i)&&(r[t]=functio
 n(){var e=function(){return n.prototype[t].apply(this,arguments)},r=function(e){return n.prototype[t].apply(this,e)};return function(){var t=this._super,n=this._superApply,s;return this._super=e,this._superApply=r,s=i.apply(this,arguments),this._super=t,this._superApply=n,s}}())}),o.prototype=e.widget.extend(u,{widgetEventPrefix:u.widgetEventPrefix||t},r,{constructor:o,namespace:a,widgetName:t,widgetBaseClass:i,widgetFullName:i}),s?(e.each(s._childConstructors,function(t,n){var r=n.prototype;e.widget(r.namespace+"."+r.widgetName,o,n._proto)}),delete s._childConstructors):n._childConstructors.push(o),e.widget.bridge(t,o)},e.widget.extend=function(n){var i=r.call(arguments,1),s=0,o=i.length,u,a;for(;s<o;s++)for(u in i[s])a=i[s][u],i[s].hasOwnProperty(u)&&a!==t&&(e.isPlainObject(a)?n[u]=e.isPlainObject(n[u])?e.widget.extend({},n[u],a):e.widget.extend({},a):n[u]=a);return n},e.widget.bridge=function(n,i){var s=i.prototype.widgetFullName;e.fn[n]=function(o){var u=typeof o=="string",a=r.c
 all(arguments,1),f=this;return o=!u&&a.length?e.widget.extend.apply(null,[o].concat(a)):o,u?this.each(function(){var r,i=e.data(this,s);if(!i)return e.error("cannot call methods on "+n+" prior to initialization; "+"attempted to call method '"+o+"'");if(!e.isFunction(i[o])||o.charAt(0)==="_")return e.error("no such method '"+o+"' for "+n+" widget instance");r=i[o].apply(i,a);if(r!==i&&r!==t)return f=r&&r.jquery?f.pushStack(r.get()):r,!1}):this.each(function(){var t=e.data(this,s);t?t.option(o||{})._init():new i(o,this)}),f}},e.Widget=function(){},e.Widget._childConstructors=[],e.Widget.prototype={widgetName:"widget",widgetEventPrefix:"",defaultElement:"<div>",options:{disabled:!1,create:null},_createWidget:function(t,r){r=e(r||this.defaultElement||this)[0],this.element=e(r),this.uuid=n++,this.eventNamespace="."+this.widgetName+this.uuid,this.options=e.widget.extend({},this.options,this._getCreateOptions(),t),this.bindings=e(),this.hoverable=e(),this.focusable=e(),r!==this&&(e.data(r,
 this.widgetName,this),e.data(r,this.widgetFullName,this),this._on(this.element,{remove:function(e){e.target===r&&this.destroy()}}),this.document=e(r.style?r.ownerDocument:r.document||r),this.window=e(this.document[0].defaultView||this.document[0].parentWindow)),this._create(),this._trigger("create",null,this._getCreateEventData()),this._init()},_getCreateOptions:e.noop,_getCreateEventData:e.noop,_create:e.noop,_init:e.noop,destroy:function(){this._destroy(),this.element.unbind(this.eventNamespace).removeData(this.widgetName).removeData(this.widgetFullName).removeData(e.camelCase(this.widgetFullName)),this.widget().unbind(this.eventNamespace).removeAttr("aria-disabled").removeClass(this.widgetFullName+"-disabled "+"ui-state-disabled"),this.bindings.unbind(this.eventNamespace),this.hoverable.removeClass("ui-state-hover"),this.focusable.removeClass("ui-state-focus")},_destroy:e.noop,widget:function(){return this.element},option:function(n,r){var i=n,s,o,u;if(arguments.length===0)return
  e.widget.extend({},this.options);if(typeof n=="string"){i={},s=n.split("."),n=s.shift();if(s.length){o=i[n]=e.widget.extend({},this.options[n]);for(u=0;u<s.length-1;u++)o[s[u]]=o[s[u]]||{},o=o[s[u]];n=s.pop();if(r===t)return o[n]===t?null:o[n];o[n]=r}else{if(r===t)return this.options[n]===t?null:this.options[n];i[n]=r}}return this._setOptions(i),this},_setOptions:function(e){var t;for(t in e)this._setOption(t,e[t]);return this},_setOption:function(e,t){return this.options[e]=t,e==="disabled"&&(this.widget().toggleClass(this.widgetFullName+"-disabled ui-state-disabled",!!t).attr("aria-disabled",t),this.hoverable.removeClass("ui-state-hover"),this.focusable.removeClass("ui-state-focus")),this},enable:function(){return this._setOption("disabled",!1)},disable:function(){return this._setOption("disabled",!0)},_on:function(t,n){var r,i=this;n?(t=r=e(t),this.bindings=this.bindings.add(t)):(n=t,t=this.element,r=this.widget()),e.each(n,function(n,s){function o(){if(i.options.disabled===!0||
 e(this).hasClass("ui-state-disabled"))return;return(typeof s=="string"?i[s]:s).apply(i,arguments)}typeof s!="string"&&(o.guid=s.guid=s.guid||o.guid||e.guid++);var u=n.match(/^(\w+)\s*(.*)$/),a=u[1]+i.eventNamespace,f=u[2];f?r.delegate(f,a,o):t.bind(a,o)})},_off:function(e,t){t=(t||"").split(" ").join(this.eventNamespace+" ")+this.eventNamespace,e.unbind(t).undelegate(t)},_delay:function(e,t){function n(){return(typeof e=="string"?r[e]:e).apply(r,arguments)}var r=this;return setTimeout(n,t||0)},_hoverable:function(t){this.hoverable=this.hoverable.add(t),this._on(t,{mouseenter:function(t){e(t.currentTarget).addClass("ui-state-hover")},mouseleave:function(t){e(t.currentTarget).removeClass("ui-state-hover")}})},_focusable:function(t){this.focusable=this.focusable.add(t),this._on(t,{focusin:function(t){e(t.currentTarget).addClass("ui-state-focus")},focusout:function(t){e(t.currentTarget).removeClass("ui-state-focus")}})},_trigger:function(t,n,r){var i,s,o=this.options[t];r=r||{},n=e.Even
 t(n),n.type=(t===this.widgetEventPrefix?t:this.widgetEventPrefix+t).toLowerCase(),n.target=this.element[0],s=n.originalEvent;if(s)for(i in s)i in n||(n[i]=s[i]);return this.element.trigger(n,r),!(e.isFunction(o)&&o.apply(this.element[0],[n].concat(r))===!1||n.isDefaultPrevented())}},e.each({show:"fadeIn",hide:"fadeOut"},function(t,n){e.Widget.prototype["_"+t]=function(r,i,s){typeof i=="string"&&(i={effect:i});var o,u=i?i===!0||typeof i=="number"?n:i.effect||n:t;i=i||{},typeof i=="number"&&(i={duration:i}),o=!e.isEmptyObject(i),i.complete=s,i.delay&&r.delay(i.delay),o&&e.effects&&(e.effects.effect[u]||e.uiBackCompat!==!1&&e.effects[u])?r[t](i):u!==t&&r[u]?r[u](i.duration,i.easing,s):r.queue(function(n){e(this)[t](),s&&s.call(r[0]),n()})}}),e.uiBackCompat!==!1&&(e.Widget.prototype._getCreateOptions=function(){return e.metadata&&e.metadata.get(this.element[0])[this.widgetName]})})(jQuery);(function(e,t){var n=!1;e(document).mouseup(function(e){n=!1}),e.widget("ui.mouse",{version:"1.9.1
 ",options:{cancel:"input,textarea,button,select,option",distance:1,delay:0},_mouseInit:function(){var t=this;this.element.bind("mousedown."+this.widgetName,function(e){return t._mouseDown(e)}).bind("click."+this.widgetName,function(n){if(!0===e.data(n.target,t.widgetName+".preventClickEvent"))return e.removeData(n.target,t.widgetName+".preventClickEvent"),n.stopImmediatePropagation(),!1}),this.started=!1},_mouseDestroy:function(){this.element.unbind("."+this.widgetName),this._mouseMoveDelegate&&e(document).unbind("mousemove."+this.widgetName,this._mouseMoveDelegate).unbind("mouseup."+this.widgetName,this._mouseUpDelegate)},_mouseDown:function(t){if(n)return;this._mouseStarted&&this._mouseUp(t),this._mouseDownEvent=t;var r=this,i=t.which===1,s=typeof this.options.cancel=="string"&&t.target.nodeName?e(t.target).closest(this.options.cancel).length:!1;if(!i||s||!this._mouseCapture(t))return!0;this.mouseDelayMet=!this.options.delay,this.mouseDelayMet||(this._mouseDelayTimer=setTimeout(fu
 nction(){r.mouseDelayMet=!0},this.options.delay));if(this._mouseDistanceMet(t)&&this._mouseDelayMet(t)){this._mouseStarted=this._mouseStart(t)!==!1;if(!this._mouseStarted)return t.preventDefault(),!0}return!0===e.data(t.target,this.widgetName+".preventClickEvent")&&e.removeData(t.target,this.widgetName+".preventClickEvent"),this._mouseMoveDelegate=function(e){return r._mouseMove(e)},this._mouseUpDelegate=function(e){return r._mouseUp(e)},e(document).bind("mousemove."+this.widgetName,this._mouseMoveDelegate).bind("mouseup."+this.widgetName,this._mouseUpDelegate),t.preventDefault(),n=!0,!0},_mouseMove:function(t){return!e.ui.ie||document.documentMode>=9||!!t.button?this._mouseStarted?(this._mouseDrag(t),t.preventDefault()):(this._mouseDistanceMet(t)&&this._mouseDelayMet(t)&&(this._mouseStarted=this._mouseStart(this._mouseDownEvent,t)!==!1,this._mouseStarted?this._mouseDrag(t):this._mouseUp(t)),!this._mouseStarted):this._mouseUp(t)},_mouseUp:function(t){return e(document).unbind("mouse
 move."+this.widgetName,this._mouseMoveDelegate).unbind("mouseup."+this.widgetName,this._mouseUpDelegate),this._mouseStarted&&(this._mouseStarted=!1,t.target===this._mouseDownEvent.target&&e.data(t.target,this.widgetName+".preventClickEvent",!0),this._mouseStop(t)),!1},_mouseDistanceMet:function(e){return Math.max(Math.abs(this._mouseDownEvent.pageX-e.pageX),Math.abs(this._mouseDownEvent.pageY-e.pageY))>=this.options.distance},_mouseDelayMet:function(e){return this.mouseDelayMet},_mouseStart:function(e){},_mouseDrag:function(e){},_mouseStop:function(e){},_mouseCapture:function(e){return!0}})})(jQuery);(function(e,t){function h(e,t,n){return[parseInt(e[0],10)*(l.test(e[0])?t/100:1),parseInt(e[1],10)*(l.test(e[1])?n/100:1)]}function p(t,n){return parseInt(e.css(t,n),10)||0}e.ui=e.ui||{};var n,r=Math.max,i=Math.abs,s=Math.round,o=/left|center|right/,u=/top|center|bottom/,a=/[\+\-]\d+%?/,f=/^\w+/,l=/%$/,c=e.fn.position;e.position={scrollbarWidth:function(){if(n!==t)return n;var r,i,s=e("
 <div style='display:block;width:50px;height:50px;overflow:hidden;'><div style='height:100px;width:auto;'></div></div>"),o=s.children()[0];return e("body").append(s),r=o.offsetWidth,s.css("overflow","scroll"),i=o.offsetWidth,r===i&&(i=s[0].clientWidth),s.remove(),n=r-i},getScrollInfo:function(t){var n=t.isWindow?"":t.element.css("overflow-x"),r=t.isWindow?"":t.element.css("overflow-y"),i=n==="scroll"||n==="auto"&&t.width<t.element[0].scrollWidth,s=r==="scroll"||r==="auto"&&t.height<t.element[0].scrollHeight;return{width:i?e.position.scrollbarWidth():0,height:s?e.position.scrollbarWidth():0}},getWithinInfo:function(t){var n=e(t||window),r=e.isWindow(n[0]);return{element:n,isWindow:r,offset:n.offset()||{left:0,top:0},scrollLeft:n.scrollLeft(),scrollTop:n.scrollTop(),width:r?n.width():n.outerWidth(),height:r?n.height():n.outerHeight()}}},e.fn.position=function(t){if(!t||!t.of)return c.apply(this,arguments);t=e.extend({},t);var n,l,d,v,m,g=e(t.of),y=e.position.getWithinInfo(t.within),b=e
 .position.getScrollInfo(y),w=g[0],E=(t.collision||"flip").split(" "),S={};return w.nodeType===9?(l=g.width(),d=g.height(),v={top:0,left:0}):e.isWindow(w)?(l=g.width(),d=g.height(),v={top:g.scrollTop(),left:g.scrollLeft()}):w.preventDefault?(t.at="left top",l=d=0,v={top:w.pageY,left:w.pageX}):(l=g.outerWidth(),d=g.outerHeight(),v=g.offset()),m=e.extend({},v),e.each(["my","at"],function(){var e=(t[this]||"").split(" "),n,r;e.length===1&&(e=o.test(e[0])?e.concat(["center"]):u.test(e[0])?["center"].concat(e):["center","center"]),e[0]=o.test(e[0])?e[0]:"center",e[1]=u.test(e[1])?e[1]:"center",n=a.exec(e[0]),r=a.exec(e[1]),S[this]=[n?n[0]:0,r?r[0]:0],t[this]=[f.exec(e[0])[0],f.exec(e[1])[0]]}),E.length===1&&(E[1]=E[0]),t.at[0]==="right"?m.left+=l:t.at[0]==="center"&&(m.left+=l/2),t.at[1]==="bottom"?m.top+=d:t.at[1]==="center"&&(m.top+=d/2),n=h(S.at,l,d),m.left+=n[0],m.top+=n[1],this.each(function(){var o,u,a=e(this),f=a.outerWidth(),c=a.outerHeight(),w=p(this,"marginLeft"),x=p(this,"margi
 nTop"),T=f+w+p(this,"marginRight")+b.width,N=c+x+p(this,"marginBottom")+b.height,C=e.extend({},m),k=h(S.my,a.outerWidth(),a.outerHeight());t.my[0]==="right"?C.left-=f:t.my[0]==="center"&&(C.left-=f/2),t.my[1]==="bottom"?C.top-=c:t.my[1]==="center"&&(C.top-=c/2),C.left+=k[0],C.top+=k[1],e.support.offsetFractions||(C.left=s(C.left),C.top=s(C.top)),o={marginLeft:w,marginTop:x},e.each(["left","top"],function(r,i){e.ui.position[E[r]]&&e.ui.position[E[r]][i](C,{targetWidth:l,targetHeight:d,elemWidth:f,elemHeight:c,collisionPosition:o,collisionWidth:T,collisionHeight:N,offset:[n[0]+k[0],n[1]+k[1]],my:t.my,at:t.at,within:y,elem:a})}),e.fn.bgiframe&&a.bgiframe(),t.using&&(u=function(e){var n=v.left-C.left,s=n+l-f,o=v.top-C.top,u=o+d-c,h={target:{element:g,left:v.left,top:v.top,width:l,height:d},element:{element:a,left:C.left,top:C.top,width:f,height:c},horizontal:s<0?"left":n>0?"right":"center",vertical:u<0?"top":o>0?"bottom":"middle"};l<f&&i(n+s)<l&&(h.horizontal="center"),d<c&&i(o+u)<d&&(h
 .vertical="middle"),r(i(n),i(s))>r(i(o),i(u))?h.important="horizontal":h.important="vertical",t.using.call(this,e,h)}),a.offset(e.extend(C,{using:u}))})},e.ui.position={fit:{left:function(e,t){var n=t.within,i=n.isWindow?n.scrollLeft:n.offset.left,s=n.width,o=e.left-t.collisionPosition.marginLeft,u=i-o,a=o+t.collisionWidth-s-i,f;t.collisionWidth>s?u>0&&a<=0?(f=e.left+u+t.collisionWidth-s-i,e.left+=u-f):a>0&&u<=0?e.left=i:u>a?e.left=i+s-t.collisionWidth:e.left=i:u>0?e.left+=u:a>0?e.left-=a:e.left=r(e.left-o,e.left)},top:function(e,t){var n=t.within,i=n.isWindow?n.scrollTop:n.offset.top,s=t.within.height,o=e.top-t.collisionPosition.marginTop,u=i-o,a=o+t.collisionHeight-s-i,f;t.collisionHeight>s?u>0&&a<=0?(f=e.top+u+t.collisionHeight-s-i,e.top+=u-f):a>0&&u<=0?e.top=i:u>a?e.top=i+s-t.collisionHeight:e.top=i:u>0?e.top+=u:a>0?e.top-=a:e.top=r(e.top-o,e.top)}},flip:{left:function(e,t){var n=t.within,r=n.offset.left+n.scrollLeft,s=n.width,o=n.isWindow?n.scrollLeft:n.offset.left,u=e.left-t.c
 ollisionPosition.marginLeft,a=u-o,f=u+t.collisionWidth-s-o,l=t.my[0]==="left"?-t.elemWidth:t.my[0]==="right"?t.elemWidth:0,c=t.at[0]==="left"?t.targetWidth:t.at[0]==="right"?-t.targetWidth:0,h=-2*t.offset[0],p,d;if(a<0){p=e.left+l+c+h+t.collisionWidth-s-r;if(p<0||p<i(a))e.left+=l+c+h}else if(f>0){d=e.left-t.collisionPosition.marginLeft+l+c+h-o;if(d>0||i(d)<f)e.left+=l+c+h}},top:function(e,t){var n=t.within,r=n.offset.top+n.scrollTop,s=n.height,o=n.isWindow?n.scrollTop:n.offset.top,u=e.top-t.collisionPosition.marginTop,a=u-o,f=u+t.collisionHeight-s-o,l=t.my[1]==="top",c=l?-t.elemHeight:t.my[1]==="bottom"?t.elemHeight:0,h=t.at[1]==="top"?t.targetHeight:t.at[1]==="bottom"?-t.targetHeight:0,p=-2*t.offset[1],d,v;a<0?(v=e.top+c+h+p+t.collisionHeight-s-r,e.top+c+h+p>a&&(v<0||v<i(a))&&(e.top+=c+h+p)):f>0&&(d=e.top-t.collisionPosition.marginTop+c+h+p-o,e.top+c+h+p>f&&(d>0||i(d)<f)&&(e.top+=c+h+p))}},flipfit:{left:function(){e.ui.position.flip.left.apply(this,arguments),e.ui.position.fit.left
 .apply(this,arguments)},top:function(){e.ui.position.flip.top.apply(this,arguments),e.ui.position.fit.top.apply(this,arguments)}}},function(){var t,n,r,i,s,o=document.getElementsByTagName("body")[0],u=document.createElement("div");t=document.createElement(o?"div":"body"),r={visibility:"hidden",width:0,height:0,border:0,margin:0,background:"none"},o&&e.extend(r,{position:"absolute",left:"-1000px",top:"-1000px"});for(s in r)t.style[s]=r[s];t.appendChild(u),n=o||document.documentElement,n.insertBefore(t,n.firstChild),u.style.cssText="position: absolute; left: 10.7432222px;",i=e(u).offset().left,e.support.offsetFractions=i>10&&i<11,t.innerHTML="",n.removeChild(t)}(),e.uiBackCompat!==!1&&function(e){var n=e.fn.position;e.fn.position=function(r){if(!r||!r.offset)return n.call(this,r);var i=r.offset.split(" "),s=r.at.split(" ");return i.length===1&&(i[1]=i[0]),/^\d/.test(i[0])&&(i[0]="+"+i[0]),/^\d/.test(i[1])&&(i[1]="+"+i[1]),s.length===1&&(/left|center|right/.test(s[0])?s[1]="center":(s[
 1]=s[0],s[0]="center")),n.call(this,e.extend(r,{at:s[0]+i[0]+" "+s[1]+i[1],offset:t}))}}(jQuery)})(jQuery);(function(e,t){var n=0,r={},i={};r.height=r.paddingTop=r.paddingBottom=r.borderTopWidth=r.borderBottomWidth="hide",i.height=i.paddingTop=i.paddingBottom=i.borderTopWidth=i.borderBottomWidth="show",e.widget("ui.accordion",{version:"1.9.1",options:{active:0,animate:{},collapsible:!1,event:"click",header:"> li > :first-child,> :not(li):even",heightStyle:"auto",icons:{activeHeader:"ui-icon-triangle-1-s",header:"ui-icon-triangle-1-e"},activate:null,beforeActivate:null},_create:function(){var t=this.accordionId="ui-accordion-"+(this.element.attr("id")||++n),r=this.options;this.prevShow=this.prevHide=e(),this.element.addClass("ui-accordion ui-widget ui-helper-reset"),this.headers=this.element.find(r.header).addClass("ui-accordion-header ui-helper-reset ui-state-default ui-corner-all"),this._hoverable(this.headers),this._focusable(this.headers),this.headers.next().addClass("ui-accordio
 n-content ui-helper-reset ui-widget-content ui-corner-bottom").hide(),!r.collapsible&&(r.active===!1||r.active==null)&&(r.active=0),r.active<0&&(r.active+=this.headers.length),this.active=this._findActive(r.active).addClass("ui-accordion-header-active ui-state-active").toggleClass("ui-corner-all ui-corner-top"),this.active.next().addClass("ui-accordion-content-active").show(),this._createIcons(),this.refresh(),this.element.attr("role","tablist"),this.headers.attr("role","tab").each(function(n){var r=e(this),i=r.attr("id"),s=r.next(),o=s.attr("id");i||(i=t+"-header-"+n,r.attr("id",i)),o||(o=t+"-panel-"+n,s.attr("id",o)),r.attr("aria-controls",o),s.attr("aria-labelledby",i)}).next().attr("role","tabpanel"),this.headers.not(this.active).attr({"aria-selected":"false",tabIndex:-1}).next().attr({"aria-expanded":"false","aria-hidden":"true"}).hide(),this.active.length?this.active.attr({"aria-selected":"true",tabIndex:0}).next().attr({"aria-expanded":"true","aria-hidden":"false"}):this.head
 ers.eq(0).attr("tabIndex",0),this._on(this.headers,{keydown:"_keydown"}),this._on(this.headers.next(),{keydown:"_panelKeyDown"}),this._setupEvents(r.event)},_getCreateEventData:function(){return{header:this.active,content:this.active.length?this.active.next():e()}},_createIcons:function(){var t=this.options.icons;t&&(e("<span>").addClass("ui-accordion-header-icon ui-icon "+t.header).prependTo(this.headers),this.active.children(".ui-accordion-header-icon").removeClass(t.header).addClass(t.activeHeader),this.headers.addClass("ui-accordion-icons"))},_destroyIcons:function(){this.headers.removeClass("ui-accordion-icons").children(".ui-accordion-header-icon").remove()},_destroy:function(){var e;this.element.removeClass("ui-accordion ui-widget ui-helper-reset").removeAttr("role"),this.headers.removeClass("ui-accordion-header ui-accordion-header-active ui-helper-reset ui-state-default ui-corner-all ui-state-active ui-state-disabled ui-corner-top").removeAttr("role").removeAttr("aria-select
 ed").removeAttr("aria-controls").removeAttr("tabIndex").each(function(){/^ui-accordion/.test(this.id)&&this.removeAttribute("id")}),this._destroyIcons(),e=this.headers.next().css("display","").removeAttr("role").removeAttr("aria-expanded").removeAttr("aria-hidden").removeAttr("aria-labelledby").removeClass("ui-helper-reset ui-widget-content ui-corner-bottom ui-accordion-content ui-accordion-content-active ui-state-disabled").each(function(){/^ui-accordion/.test(this.id)&&this.removeAttribute("id")}),this.options.heightStyle!=="content"&&e.css("height","")},_setOption:function(e,t){if(e==="active"){this._activate(t);return}e==="event"&&(this.options.event&&this._off(this.headers,this.options.event),this._setupEvents(t)),this._super(e,t),e==="collapsible"&&!t&&this.options.active===!1&&this._activate(0),e==="icons"&&(this._destroyIcons(),t&&this._createIcons()),e==="disabled"&&this.headers.add(this.headers.next()).toggleClass("ui-state-disabled",!!t)},_keydown:function(t){if(t.altKey|
 |t.ctrlKey)return;var n=e.ui.keyCode,r=this.headers.length,i=this.headers.index(t.target),s=!1;switch(t.keyCode){case n.RIGHT:case n.DOWN:s=this.headers[(i+1)%r];break;case n.LEFT:case n.UP:s=this.headers[(i-1+r)%r];break;case n.SPACE:case n.ENTER:this._eventHandler(t);break;case n.HOME:s=this.headers[0];break;case n.END:s=this.headers[r-1]}s&&(e(t.target).attr("tabIndex",-1),e(s).attr("tabIndex",0),s.focus(),t.preventDefault())},_panelKeyDown:function(t){t.keyCode===e.ui.keyCode.UP&&t.ctrlKey&&e(t.currentTarget).prev().focus()},refresh:function(){var t,n,r=this.options.heightStyle,i=this.element.parent();r==="fill"?(e.support.minHeight||(n=i.css("overflow"),i.css("overflow","hidden")),t=i.height(),this.element.siblings(":visible").each(function(){var n=e(this),r=n.css("position");if(r==="absolute"||r==="fixed")return;t-=n.outerHeight(!0)}),n&&i.css("overflow",n),this.headers.each(function(){t-=e(this).outerHeight(!0)}),this.headers.next().each(function(){e(this).height(Math.max(0,t
 -e(this).innerHeight()+e(this).height()))}).css("overflow","auto")):r==="auto"&&(t=0,this.headers.next().each(function(){t=Math.max(t,e(this).height("").height())}).height(t))},_activate:function(t){var n=this._findActive(t)[0];if(n===this.active[0])return;n=n||this.active[0],this._eventHandler({target:n,currentTarget:n,preventDefault:e.noop})},_findActive:function(t){return typeof t=="number"?this.headers.eq(t):e()},_setupEvents:function(t){var n={};if(!t)return;e.each(t.split(" "),function(e,t){n[t]="_eventHandler"}),this._on(this.headers,n)},_eventHandler:function(t){var n=this.options,r=this.active,i=e(t.currentTarget),s=i[0]===r[0],o=s&&n.collapsible,u=o?e():i.next(),a=r.next(),f={oldHeader:r,oldPanel:a,newHeader:o?e():i,newPanel:u};t.preventDefault();if(s&&!n.collapsible||this._trigger("beforeActivate",t,f)===!1)return;n.active=o?!1:this.headers.index(i),this.active=s?e():i,this._toggle(f),r.removeClass("ui-accordion-header-active ui-state-active"),n.icons&&r.children(".ui-acc
 ordion-header-icon").removeClass(n.icons.activeHeader).addClass(n.icons.header),s||(i.removeClass("ui-corner-all").addClass("ui-accordion-header-active ui-state-active ui-corner-top"),n.icons&&i.children(".ui-accordion-header-icon").removeClass(n.icons.header).addClass(n.icons.activeHeader),i.next().addClass("ui-accordion-content-active"))},_toggle:function(t){var n=t.newPanel,r=this.prevShow.length?this.prevShow:t.oldPanel;this.prevShow.add(this.prevHide).stop(!0,!0),this.prevShow=n,this.prevHide=r,this.options.animate?this._animate(n,r,t):(r.hide(),n.show(),this._toggleComplete(t)),r.attr({"aria-expanded":"false","aria-hidden":"true"}),r.prev().attr("aria-selected","false"),n.length&&r.length?r.prev().attr("tabIndex",-1):n.length&&this.headers.filter(function(){return e(this).attr("tabIndex")===0}).attr("tabIndex",-1),n.attr({"aria-expanded":"true","aria-hidden":"false"}).prev().attr({"aria-selected":"true",tabIndex:0})},_animate:function(e,t,n){var s,o,u,a=this,f=0,l=e.length&&(!
 t.length||e.index()<t.index()),c=this.options.animate||{},h=l&&c.down||c,p=function(){a._toggleComplete(n)};typeof h=="number"&&(u=h),typeof h=="string"&&(o=h),o=o||h.easing||c.easing,u=u||h.duration||c.duration;if(!t.length)return e.animate(i,u,o,p);if(!e.length)return t.animate(r,u,o,p);s=e.show().outerHeight(),t.animate(r,{duration:u,easing:o,step:function(e,t){t.now=Math.round(e)}}),e.hide().animate(i,{duration:u,easing:o,complete:p,step:function(e,n){n.now=Math.round(e),n.prop!=="height"?f+=n.now:a.options.heightStyle!=="content"&&(n.now=Math.round(s-t.outerHeight()-f),f=0)}})},_toggleComplete:function(e){var t=e.oldPanel;t.removeClass("ui-accordion-content-active").prev().removeClass("ui-corner-top").addClass("ui-corner-all"),t.length&&(t.parent()[0].className=t.parent()[0].className),this._trigger("activate",null,e)}}),e.uiBackCompat!==!1&&(function(e,t){e.extend(t.options,{navigation:!1,navigationFilter:function(){return this.href.toLowerCase()===location.href.toLowerCase()}
 });var n=t._create;t._create=function(){if(this.options.navigation){var t=this,r=this.element.find(this.options.header),i=r.next(),s=r.add(i).find("a").filter(this.options.navigationFilter)[0];s&&r.add(i).each(function(n){if(e.contains(this,s))return t.options.active=Math.floor(n/2),!1})}n.call(this)}}(jQuery,jQuery.ui.accordion.prototype),function(e,t){e.extend(t.options,{heightStyle:null,autoHeight:!0,clearStyle:!1,fillSpace:!1});var n=t._create,r=t._setOption;e.extend(t,{_create:function(){this.options.heightStyle=this.options.heightStyle||this._mergeHeightStyle(),n.call(this)},_setOption:function(e){if(e==="autoHeight"||e==="clearStyle"||e==="fillSpace")this.options.heightStyle=this._mergeHeightStyle();r.apply(this,arguments)},_mergeHeightStyle:function(){var e=this.options;if(e.fillSpace)return"fill";if(e.clearStyle)return"content";if(e.autoHeight)return"auto"}})}(jQuery,jQuery.ui.accordion.prototype),function(e,t){e.extend(t.options.icons,{activeHeader:null,headerSelected:"ui-
 icon-triangle-1-s"});var n=t._createIcons;t._createIcons=function(){this.options.icons&&(this.options.icons.activeHeader=this.options.icons.activeHeader||this.options.icons.headerSelected),n.call(this)}}(jQuery,jQuery.ui.accordion.prototype),function(e,t){t.activate=t._activate;var n=t._findActive;t._findActive=function(e){return e===-1&&(e=!1),e&&typeof e!="number"&&(e=this.headers.index(this.headers.filter(e)),e===-1&&(e=!1)),n.call(this,e)}}(jQuery,jQuery.ui.accordion.prototype),jQuery.ui.accordion.prototype.resize=jQuery.ui.accordion.prototype.refresh,function(e,t){e.extend(t.options,{change:null,changestart:null});var n=t._trigger;t._trigger=function(e,t,r){var i=n.apply(this,arguments);return i?(e==="beforeActivate"?i=n.call(this,"changestart",t,{oldHeader:r.oldHeader,oldContent:r.oldPanel,newHeader:r.newHeader,newContent:r.newPanel}):e==="activate"&&(i=n.call(this,"change",t,{oldHeader:r.oldHeader,oldContent:r.oldPanel,newHeader:r.newHeader,newContent:r.newPanel})),i):!1}}(jQ
 uery,jQuery.ui.accordion.prototype),function(e,t){e.extend(t.options,{animate:null,animated:"slide"});var n=t._create;t._create=function(){var e=this.options;e.animate===null&&(e.animated?e.animated==="slide"?e.animate=300:e.animated==="bounceslide"?e.animate={duration:200,down:{easing:"easeOutBounce",duration:1e3}}:e.animate=e.animated:e.animate=!1),n.call(this)}}(jQuery,jQuery.ui.accordion.prototype))})(jQuery);(function(e,t){var n=0;e.widget("ui.autocomplete",{version:"1.9.1",defaultElement:"<input>",options:{appendTo:"body",autoFocus:!1,delay:300,minLength:1,position:{my:"left top",at:"left bottom",collision:"none"},source:null,change:null,close:null,focus:null,open:null,response:null,search:null,select:null},pending:0,_create:function(){var t,n,r;this.isMultiLine=this._isMultiLine(),this.valueMethod=this.element[this.element.is("input,textarea")?"val":"text"],this.isNewMenu=!0,this.element.addClass("ui-autocomplete-input").attr("autocomplete","off"),this._on(this.element,{keydo
 wn:function(i){if(this.element.prop("readOnly")){t=!0,r=!0,n=!0;return}t=!1,r=!1,n=!1;var s=e.ui.keyCode;switch(i.keyCode){case s.PAGE_UP:t=!0,this._move("previousPage",i);break;case s.PAGE_DOWN:t=!0,this._move("nextPage",i);break;case s.UP:t=!0,this._keyEvent("previous",i);break;case s.DOWN:t=!0,this._keyEvent("next",i);break;case s.ENTER:case s.NUMPAD_ENTER:this.menu.active&&(t=!0,i.preventDefault(),this.menu.select(i));break;case s.TAB:this.menu.active&&this.menu.select(i);break;case s.ESCAPE:this.menu.element.is(":visible")&&(this._value(this.term),this.close(i),i.preventDefault());break;default:n=!0,this._searchTimeout(i)}},keypress:function(r){if(t){t=!1,r.preventDefault();return}if(n)return;var i=e.ui.keyCode;switch(r.keyCode){case i.PAGE_UP:this._move("previousPage",r);break;case i.PAGE_DOWN:this._move("nextPage",r);break;case i.UP:this._keyEvent("previous",r);break;case i.DOWN:this._keyEvent("next",r)}},input:function(e){if(r){r=!1,e.preventDefault();return}this._searchTime
 out(e)},focus:function(){this.selectedItem=null,this.previous=this._value()},blur:function(e){if(this.cancelBlur){delete this.cancelBlur;return}clearTimeout(this.searching),this.close(e),this._change(e)}}),this._initSource(),this.menu=e("<ul>").addClass("ui-autocomplete").appendTo(this.document.find(this.options.appendTo||"body")[0]).menu({input:e(),role:null}).zIndex(this.element.zIndex()+1).hide().data("menu"),this._on(this.menu.element,{mousedown:function(t){t.preventDefault(),this.cancelBlur=!0,this._delay(function(){delete this.cancelBlur});var n=this.menu.element[0];e(t.target).closest(".ui-menu-item").length||this._delay(function(){var t=this;this.document.one("mousedown",function(r){r.target!==t.element[0]&&r.target!==n&&!e.contains(n,r.target)&&t.close()})})},menufocus:function(t,n){if(this.isNewMenu){this.isNewMenu=!1;if(t.originalEvent&&/^mouse/.test(t.originalEvent.type)){this.menu.blur(),this.document.one("mousemove",function(){e(t.target).trigger(t.originalEvent)});ret
 urn}}var r=n.item.data("ui-autocomplete-item")||n.item.data("item.autocomplete");!1!==this._trigger("focus",t,{item:r})?t.originalEvent&&/^key/.test(t.originalEvent.type)&&this._value(r.value):this.liveRegion.text(r.value)},menuselect:function(e,t){var n=t.item.data("ui-autocomplete-item")||t.item.data("item.autocomplete"),r=this.previous;this.element[0]!==this.document[0].activeElement&&(this.element.focus(),this.previous=r,this._delay(function(){this.previous=r,this.selectedItem=n})),!1!==this._trigger("select",e,{item:n})&&this._value(n.value),this.term=this._value(),this.close(e),this.selectedItem=n}}),this.liveRegion=e("<span>",{role:"status","aria-live":"polite"}).addClass("ui-helper-hidden-accessible").insertAfter(this.element),e.fn.bgiframe&&this.menu.element.bgiframe(),this._on(this.window,{beforeunload:function(){this.element.removeAttr("autocomplete")}})},_destroy:function(){clearTimeout(this.searching),this.element.removeClass("ui-autocomplete-input").removeAttr("autocom
 plete"),this.menu.element.remove(),this.liveRegion.remove()},_setOption:function(e,t){this._super(e,t),e==="source"&&this._initSource(),e==="appendTo"&&this.menu.element.appendTo(this.document.find(t||"body")[0]),e==="disabled"&&t&&this.xhr&&this.xhr.abort()},_isMultiLine:function(){return this.element.is("textarea")?!0:this.element.is("input")?!1:this.element.prop("isContentEditable")},_initSource:function(){var t,n,r=this;e.isArray(this.options.source)?(t=this.options.source,this.source=function(n,r){r(e.ui.autocomplete.filter(t,n.term))}):typeof this.options.source=="string"?(n=this.options.source,this.source=function(t,i){r.xhr&&r.xhr.abort(),r.xhr=e.ajax({url:n,data:t,dataType:"json",success:function(e){i(e)},error:function(){i([])}})}):this.source=this.options.source},_searchTimeout:function(e){clearTimeout(this.searching),this.searching=this._delay(function(){this.term!==this._value()&&(this.selectedItem=null,this.search(null,e))},this.options.delay)},search:function(e,t){e=e
 !=null?e:this._value(),this.term=this._value();if(e.length<this.options.minLength)return this.close(t);if(this._trigger("search",t)===!1)return;return this._search(e)},_search:function(e){this.pending++,this.element.addClass("ui-autocomplete-loading"),this.cancelSearch=!1,this.source({term:e},this._response())},_response:function(){var e=this,t=++n;return function(r){t===n&&e.__response(r),e.pending--,e.pending||e.element.removeClass("ui-autocomplete-loading")}},__response:function(e){e&&(e=this._normalize(e)),this._trigger("response",null,{content:e}),!this.options.disabled&&e&&e.length&&!this.cancelSearch?(this._suggest(e),this._trigger("open")):this._close()},close:function(e){this.cancelSearch=!0,this._close(e)},_close:function(e){this.menu.element.is(":visible")&&(this.menu.element.hide(),this.menu.blur(),this.isNewMenu=!0,this._trigger("close",e))},_change:function(e){this.previous!==this._value()&&this._trigger("change",e,{item:this.selectedItem})},_normalize:function(t){retu
 rn t.length&&t[0].label&&t[0].value?t:e.map(t,function(t){return typeof t=="string"?{label:t,value:t}:e.extend({label:t.label||t.value,value:t.value||t.label},t)})},_suggest:function(t){var n=this.menu.element.empty().zIndex(this.element.zIndex()+1);this._renderMenu(n,t),this.menu.refresh(),n.show(),this._resizeMenu(),n.position(e.extend({of:this.element},this.options.position)),this.options.autoFocus&&this.menu.next()},_resizeMenu:function(){var e=this.menu.element;e.outerWidth(Math.max(e.width("").outerWidth()+1,this.element.outerWidth()))},_renderMenu:function(t,n){var r=this;e.each(n,function(e,n){r._renderItemData(t,n)})},_renderItemData:function(e,t){return this._renderItem(e,t).data("ui-autocomplete-item",t)},_renderItem:function(t,n){return e("<li>").append(e("<a>").text(n.label)).appendTo(t)},_move:function(e,t){if(!this.menu.element.is(":visible")){this.search(null,t);return}if(this.menu.isFirstItem()&&/^previous/.test(e)||this.menu.isLastItem()&&/^next/.test(e)){this._val
 ue(this.term),this.menu.blur();return}this.menu[e](t)},widget:function(){return this.menu.element},_value:function(){return this.valueMethod.apply(this.element,arguments)},_keyEvent:function(e,t){if(!this.isMultiLine||this.menu.element.is(":visible"))this._move(e,t),t.preventDefault()}}),e.extend(e.ui.autocomplete,{escapeRegex:function(e){return e.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g,"\\$&")},filter:function(t,n){var r=new RegExp(e.ui.autocomplete.escapeRegex(n),"i");return e.grep(t,function(e){return r.test(e.label||e.value||e)})}}),e.widget("ui.autocomplete",e.ui.autocomplete,{options:{messages:{noResults:"No search results.",results:function(e){return e+(e>1?" results are":" result is")+" available, use up and down arrow keys to navigate."}}},__response:function(e){var t;this._superApply(arguments);if(this.options.disabled||this.cancelSearch)return;e&&e.length?t=this.options.messages.results(e.length):t=this.options.messages.noResults,this.liveRegion.text(t)}})})(jQuery);(functi
 on(e,t){var n,r,i,s,o="ui-button ui-widget ui-state-default ui-corner-all",u="ui-state-hover ui-state-active ",a="ui-button-icons-only ui-button-icon-only ui-button-text-icons ui-button-text-icon-primary ui-button-text-icon-secondary ui-button-text-only",f=function(){var t=e(this).find(":ui-button");setTimeout(function(){t.button("refresh")},1)},l=function(t){var n=t.name,r=t.form,i=e([]);return n&&(r?i=e(r).find("[name='"+n+"']"):i=e("[name='"+n+"']",t.ownerDocument).filter(function(){return!this.form})),i};e.widget("ui.button",{version:"1.9.1",defaultElement:"<button>",options:{disabled:null,text:!0,label:null,icons:{primary:null,secondary:null}},_create:function(){this.element.closest("form").unbind("reset"+this.eventNamespace).bind("reset"+this.eventNamespace,f),typeof this.options.disabled!="boolean"?this.options.disabled=!!this.element.prop("disabled"):this.element.prop("disabled",this.options.disabled),this._determineButtonType(),this.hasTitle=!!this.buttonElement.attr("title
 ");var t=this,u=this.options,a=this.type==="checkbox"||this.type==="radio",c="ui-state-hover"+(a?"":" ui-state-active"),h="ui-state-focus";u.label===null&&(u.label=this.type==="input"?this.buttonElement.val():this.buttonElement.html()),this.buttonElement.addClass(o).attr("role","button").bind("mouseenter"+this.eventNamespace,function(){if(u.disabled)return;e(this).addClass("ui-state-hover"),this===n&&e(this).addClass("ui-state-active")}).bind("mouseleave"+this.eventNamespace,function(){if(u.disabled)return;e(this).removeClass(c)}).bind("click"+this.eventNamespace,function(e){u.disabled&&(e.preventDefault(),e.stopImmediatePropagation())}),this.element.bind("focus"+this.eventNamespace,function(){t.buttonElement.addClass(h)}).bind("blur"+this.eventNamespace,function(){t.buttonElement.removeClass(h)}),a&&(this.element.bind("change"+this.eventNamespace,function(){if(s)return;t.refresh()}),this.buttonElement.bind("mousedown"+this.eventNamespace,function(e){if(u.disabled)return;s=!1,r=e.pa
 geX,i=e.pageY}).bind("mouseup"+this.eventNamespace,function(e){if(u.disabled)return;if(r!==e.pageX||i!==e.pageY)s=!0})),this.type==="checkbox"?this.buttonElement.bind("click"+this.eventNamespace,function(){if(u.disabled||s)return!1;e(this).toggleClass("ui-state-active"),t.buttonElement.attr("aria-pressed",t.element[0].checked)}):this.type==="radio"?this.buttonElement.bind("click"+this.eventNamespace,function(){if(u.disabled||s)return!1;e(this).addClass("ui-state-active"),t.buttonElement.attr("aria-pressed","true");var n=t.element[0];l(n).not(n).map(function(){return e(this).button("widget")[0]}).removeClass("ui-state-active").attr("aria-pressed","false")}):(this.buttonElement.bind("mousedown"+this.eventNamespace,function(){if(u.disabled)return!1;e(this).addClass("ui-state-active"),n=this,t.document.one("mouseup",function(){n=null})}).bind("mouseup"+this.eventNamespace,function(){if(u.disabled)return!1;e(this).removeClass("ui-state-active")}).bind("keydown"+this.eventNamespace,functi
 on(t){if(u.disabled)return!1;(t.keyCode===e.ui.keyCode.SPACE||t.keyCode===e.ui.keyCode.ENTER)&&e(this).addClass("ui-state-active")}).bind("keyup"+this.eventNamespace,function(){e(this).removeClass("ui-state-active")}),this.buttonElement.is("a")&&this.buttonElement.keyup(function(t){t.keyCode===e.ui.keyCode.SPACE&&e(this).click()})),this._setOption("disabled",u.disabled),this._resetButton()},_determineButtonType:function(){var e,t,n;this.element.is("[type=checkbox]")?this.type="checkbox":this.element.is("[type=radio]")?this.type="radio":this.element.is("input")?this.type="input":this.type="button",this.type==="checkbox"||this.type==="radio"?(e=this.element.parents().last(),t="label[for='"+this.element.attr("id")+"']",this.buttonElement=e.find(t),this.buttonElement.length||(e=e.length?e.siblings():this.element.siblings(),this.buttonElement=e.filter(t),this.buttonElement.length||(this.buttonElement=e.find(t))),this.element.addClass("ui-helper-hidden-accessible"),n=this.element.is(":che
 cked"),n&&this.buttonElement.addClass("ui-state-active"),this.buttonElement.prop("aria-pressed",n)):this.buttonElement=this.element},widget:function(){return this.buttonElement},_destroy:function(){this.element.removeClass("ui-helper-hidden-accessible"),this.buttonElement.removeClass(o+" "+u+" "+a).removeAttr("role").removeAttr("aria-pressed").html(this.buttonElement.find(".ui-button-text").html()),this.hasTitle||this.buttonElement.removeAttr("title")},_setOption:function(e,t){this._super(e,t);if(e==="disabled"){t?this.element.prop("disabled",!0):this.element.prop("disabled",!1);return}this._resetButton()},refresh:function(){var t=this.element.is(":disabled")||this.element.hasClass("ui-button-disabled");t!==this.options.disabled&&this._setOption("disabled",t),this.type==="radio"?l(this.element[0]).each(function(){e(this).is(":checked")?e(this).button("widget").addClass("ui-state-active").attr("aria-pressed","true"):e(this).button("widget").removeClass("ui-state-active").attr("aria-p
 ressed","false")}):this.type==="checkbox"&&(this.element.is(":checked")?this.buttonElement.addClass("ui-state-active").attr("aria-pressed","true"):this.buttonElement.removeClass("ui-state-active").attr("aria-pressed","false"))},_resetButton:function(){if(this.type==="input"){this.options.label&&this.element.val(this.options.label);return}var t=this.buttonElement.removeClass(a),n=e("<span></span>",this.document[0]).addClass("ui-button-text").html(this.options.label).appendTo(t.empty()).text(),r=this.options.icons,i=r.primary&&r.secondary,s=[];r.primary||r.secondary?(this.options.text&&s.push("ui-button-text-icon"+(i?"s":r.primary?"-primary":"-secondary")),r.primary&&t.prepend("<span class='ui-button-icon-primary ui-icon "+r.primary+"'></span>"),r.secondary&&t.append("<span class='ui-button-icon-secondary ui-icon "+r.secondary+"'></span>"),this.options.text||(s.push(i?"ui-button-icons-only":"ui-button-icon-only"),this.hasTitle||t.attr("title",e.trim(n)))):s.push("ui-button-text-only")
 ,t.addClass(s.join(" "))}}),e.widget("ui.buttonset",{version:"1.9.1",options:{items:"button, input[type=button], input[type=submit], input[type=reset], input[type=checkbox], input[type=radio], a, :data(button)"},_create:function(){this.element.addClass("ui-buttonset")},_init:function(){this.refresh()},_setOption:function(e,t){e==="disabled"&&this.buttons.button("option",e,t),this._super(e,t)},refresh:function(){var t=this.element.css("direction")==="rtl";this.buttons=this.element.find(this.options.items).filter(":ui-button").button("refresh").end().not(":ui-button").button().end().map(function(){return e(this).button("widget")[0]}).removeClass("ui-corner-all ui-corner-left ui-corner-right").filter(":first").addClass(t?"ui-corner-right":"ui-corner-left").end().filter(":last").addClass(t?"ui-corner-left":"ui-corner-right").end().end()},_destroy:function(){this.element.removeClass("ui-buttonset"),this.buttons.map(function(){return e(this).button("widget")[0]}).removeClass("ui-corner-le
 ft ui-corner-right").end().button("destroy")}})})(jQuery);(function($,undefined){function Datepicker(){this.debug=!1,this._curInst=null,this._keyEvent=!1,this._disabledInputs=[],this._datepickerShowing=!1,this._inDialog=!1,this._mainDivId="ui-datepicker-div",this._inlineClass="ui-datepicker-inline",this._appendClass="ui-datepicker-append",this._triggerClass="ui-datepicker-trigger",this._dialogClass="ui-datepicker-dialog",this._disableClass="ui-datepicker-disabled",this._unselectableClass="ui-datepicker-unselectable",this._currentClass="ui-datepicker-current-day",this._dayOverClass="ui-datepicker-days-cell-over",this.regional=[],this.regional[""]={closeText:"Done",prevText:"Prev",nextText:"Next",currentText:"Today",monthNames:["January","February","March","April","May","June","July","August","September","October","November","December"],monthNamesShort:["Jan","Feb","Mar","Apr","May","Jun","Jul","Aug","Sep","Oct","Nov","Dec"],dayNames:["Sunday","Monday","Tuesday","Wednesday","Thursday"
 ,"Friday","Saturday"],dayNamesShort:["Sun","Mon","Tue","Wed","Thu","Fri","Sat"],dayNamesMin:["Su","Mo","Tu","We","Th","Fr","Sa"],weekHeader:"Wk",dateFormat:"mm/dd/yy",firstDay:0,isRTL:!1,showMonthAfterYear:!1,yearSuffix:""},this._defaults={showOn:"focus",showAnim:"fadeIn",showOptions:{},defaultDate:null,appendText:"",buttonText:"...",buttonImage:"",buttonImageOnly:!1,hideIfNoPrevNext:!1,navigationAsDateFormat:!1,gotoCurrent:!1,changeMonth:!1,changeYear:!1,yearRange:"c-10:c+10",showOtherMonths:!1,selectOtherMonths:!1,showWeek:!1,calculateWeek:this.iso8601Week,shortYearCutoff:"+10",minDate:null,maxDate:null,duration:"fast",beforeShowDay:null,beforeShow:null,onSelect:null,onChangeMonthYear:null,onClose:null,numberOfMonths:1,showCurrentAtPos:0,stepMonths:1,stepBigMonths:12,altField:"",altFormat:"",constrainInput:!0,showButtonPanel:!1,autoSize:!1,disabled:!1},$.extend(this._defaults,this.regional[""]),this.dpDiv=bindHover($('<div id="'+this._mainDivId+'" class="ui-datepicker ui-widget ui
 -widget-content ui-helper-clearfix ui-corner-all"></div>'))}function bindHover(e){var t="button, .ui-datepicker-prev, .ui-datepicker-next, .ui-datepicker-calendar td a";return e.delegate(t,"mouseout",function(){$(this).removeClass("ui-state-hover"),this.className.indexOf("ui-datepicker-prev")!=-1&&$(this).removeClass("ui-datepicker-prev-hover"),this.className.indexOf("ui-datepicker-next")!=-1&&$(this).removeClass("ui-datepicker-next-hover")}).delegate(t,"mouseover",function(){$.datepicker._isDisabledDatepicker(instActive.inline?e.parent()[0]:instActive.input[0])||($(this).parents(".ui-datepicker-calendar").find("a").removeClass("ui-state-hover"),$(this).addClass("ui-state-hover"),this.className.indexOf("ui-datepicker-prev")!=-1&&$(this).addClass("ui-datepicker-prev-hover"),this.className.indexOf("ui-datepicker-next")!=-1&&$(this).addClass("ui-datepicker-next-hover"))})}function extendRemove(e,t){$.extend(e,t);for(var n in t)if(t[n]==null||t[n]==undefined)e[n]=t[n];return e}$.extend(
 $.ui,{datepicker:{version:"1.9.1"}});var PROP_NAME="datepicker",dpuuid=(new Date).getTime(),instActive;$.extend(Datepicker.prototype,{markerClassName:"hasDatepicker",maxRows:4,log:function(){this.debug&&console.log.apply("",arguments)},_widgetDatepicker:function(){return this.dpDiv},setDefaults:function(e){return extendRemove(this._defaults,e||{}),this},_attachDatepicker:function(target,settings){var inlineSettings=null;for(var attrName in this._defaults){var attrValue=target.getAttribute("date:"+attrName);if(attrValue){inlineSettings=inlineSettings||{};try{inlineSettings[attrName]=eval(attrValue)}catch(err){inlineSettings[attrName]=attrValue}}}var nodeName=target.nodeName.toLowerCase(),inline=nodeName=="div"||nodeName=="span";target.id||(this.uuid+=1,target.id="dp"+this.uuid);var inst=this._newInst($(target),inline);inst.settings=$.extend({},settings||{},inlineSettings||{}),nodeName=="input"?this._connectDatepicker(target,inst):inline&&this._inlineDatepicker(target,inst)},_newInst:
 function(e,t){var n=e[0].id.replace(/([^A-Za-z0-9_-])/g,"\\\\$1");return{id:n,input:e,selectedDay:0,selectedMonth:0,selectedYear:0,drawMonth:0,drawYear:0,inline:t,dpDiv:t?bindHover($('<div class="'+this._inlineClass+' ui-datepicker ui-widget ui-widget-content ui-helper-clearfix ui-corner-all"></div>')):this.dpDiv}},_connectDatepicker:function(e,t){var n=$(e);t.append=$([]),t.trigger=$([]);if(n.hasClass(this.markerClassName))return;this._attachments(n,t),n.addClass(this.markerClassName).keydown(this._doKeyDown).keypress(this._doKeyPress).keyup(this._doKeyUp).bind("setData.datepicker",function(e,n,r){t.settings[n]=r}).bind("getData.datepicker",function(e,n){return this._get(t,n)}),this._autoSize(t),$.data(e,PROP_NAME,t),t.settings.disabled&&this._disableDatepicker(e)},_attachments:function(e,t){var n=this._get(t,"appendText"),r=this._get(t,"isRTL");t.append&&t.append.remove(),n&&(t.append=$('<span class="'+this._appendClass+'">'+n+"</span>"),e[r?"before":"after"](t.append)),e.unbind("
 focus",this._showDatepicker),t.trigger&&t.trigger.remove();var i=this._get(t,"showOn");(i=="focus"||i=="both")&&e.focus(this._showDatepicker);if(i=="button"||i=="both"){var s=this._get(t,"buttonText"),o=this._get(t,"buttonImage");t.trigger=$(this._get(t,"buttonImageOnly")?$("<img/>").addClass(this._triggerClass).attr({src:o,alt:s,title:s}):$('<button type="button"></button>').addClass(this._triggerClass).html(o==""?s:$("<img/>").attr({src:o,alt:s,title:s}))),e[r?"before":"after"](t.trigger),t.trigger.click(function(){return $.datepicker._datepickerShowing&&$.datepicker._lastInput==e[0]?$.datepicker._hideDatepicker():$.datepicker._datepickerShowing&&$.datepicker._lastInput!=e[0]?($.datepicker._hideDatepicker(),$.datepicker._showDatepicker(e[0])):$.datepicker._showDatepicker(e[0]),!1})}},_autoSize:function(e){if(this._get(e,"autoSize")&&!e.inline){var t=new Date(2009,11,20),n=this._get(e,"dateFormat");if(n.match(/[DM]/)){var r=function(e){var t=0,n=0;for(var r=0;r<e.length;r++)e[r].le
 ngth>t&&(t=e[r].length,n=r);return n};t.setMonth(r(this._get(e,n.match(/MM/)?"monthNames":"monthNamesShort"))),t.setDate(r(this._get(e,n.match(/DD/)?"dayNames":"dayNamesShort"))+20-t.getDay())}e.input.attr("size",this._formatDate(e,t).length)}},_inlineDatepicker:function(e,t){var n=$(e);if(n.hasClass(this.markerClassName))return;n.addClass(this.markerClassName).append(t.dpDiv).bind("setData.datepicker",function(e,n,r){t.settings[n]=r}).bind("getData.datepicker",function(e,n){return this._get(t,n)}),$.data(e,PROP_NAME,t),this._setDate(t,this._getDefaultDate(t),!0),this._updateDatepicker(t),this._updateAlternate(t),t.settings.disabled&&this._disableDatepicker(e),t.dpDiv.css("display","block")},_dialogDatepicker:function(e,t,n,r,i){var s=this._dialogInst;if(!s){this.uuid+=1;var o="dp"+this.uuid;this._dialogInput=$('<input type="text" id="'+o+'" style="position: absolute; top: -100px; width: 0px;"/>'),this._dialogInput.keydown(this._doKeyDown),$("body").append(this._dialogInput),s=this.
 _dialogInst=this._newInst(this._dialogInput,!1),s.settings={},$.data(this._dialogInput[0],PROP_NAME,s)}extendRemove(s.settings,r||{}),t=t&&t.constructor==Date?this._formatDate(s,t):t,this._dialogInput.val(t),this._pos=i?i.length?i:[i.pageX,i.pageY]:null;if(!this._pos){var u=document.documentElement.clientWidth,a=document.documentElement.clientHeight,f=document.documentElement.scrollLeft||document.body.scrollLeft,l=document.documentElement.scrollTop||document.body.scrollTop;this._pos=[u/2-100+f,a/2-150+l]}return this._dialogInput.css("left",this._pos[0]+20+"px").css("top",this._pos[1]+"px"),s.settings.onSelect=n,this._inDialog=!0,this.dpDiv.addClass(this._dialogClass),this._showDatepicker(this._dialogInput[0]),$.blockUI&&$.blockUI(this.dpDiv),$.data(this._dialogInput[0],PROP_NAME,s),this},_destroyDatepicker:function(e){var t=$(e),n=$.data(e,PROP_NAME);if(!t.hasClass(this.markerClassName))return;var r=e.nodeName.toLowerCase();$.removeData(e,PROP_NAME),r=="input"?(n.append.remove(),n.t
 rigger.remove(),t.removeClass(this.markerClassName).unbind("focus",this._showDatepicker).unbind("keydown",this._doKeyDown).unbind("keypress",this._doKeyPress).unbind("keyup",this._doKeyUp)):(r=="div"||r=="span")&&t.removeClass(this.markerClassName).empty()},_enableDatepicker:function(e){var t=$(e),n=$.data(e,PROP_NAME);if(!t.hasClass(this.markerClassName))return;var r=e.nodeName.toLowerCase();if(r=="input")e.disabled=!1,n.trigger.filter("button").each(function(){this.disabled=!1}).end().filter("img").css({opacity:"1.0",cursor:""});else if(r=="div"||r=="span"){var i=t.children("."+this._inlineClass);i.children().removeClass("ui-state-disabled"),i.find("select.ui-datepicker-month, select.ui-datepicker-year").prop("disabled",!1)}this._disabledInputs=$.map(this._disabledInputs,function(t){return t==e?null:t})},_disableDatepicker:function(e){var t=$(e),n=$.data(e,PROP_NAME);if(!t.hasClass(this.markerClassName))return;var r=e.nodeName.toLowerCase();if(r=="input")e.disabled=!0,n.trigger.fi
 lter("button").each(function(){this.disabled=!0}).end().filter("img").css({opacity:"0.5",cursor:"default"});else if(r=="div"||r=="span"){var i=t.children("."+this._inlineClass);i.children().addClass("ui-state-disabled"),i.find("select.ui-datepicker-month, select.ui-datepicker-year").prop("disabled",!0)}this._disabledInputs=$.map(this._disabledInputs,function(t){return t==e?null:t}),this._disabledInputs[this._disabledInputs.length]=e},_isDisabledDatepicker:function(e){if(!e)return!1;for(var t=0;t<this._disabledInputs.length;t++)if(this._disabledInputs[t]==e)return!0;return!1},_getInst:function(e){try{return $.data(e,PROP_NAME)}catch(t){throw"Missing instance data for this datepicker"}},_optionDatepicker:function(e,t,n){var r=this._getInst(e);if(arguments.length==2&&typeof t=="string")return t=="defaults"?$.extend({},$.datepicker._defaults):r?t=="all"?$.extend({},r.settings):this._get(r,t):null;var i=t||{};typeof t=="string"&&(i={},i[t]=n);if(r){this._curInst==r&&this._hideDatepicker(
 );var s=this._getDateDatepicker(e,!0),o=this._getMinMaxDate(r,"min"),u=this._getMinMaxDate(r,"max");extendRemove(r.settings,i),o!==null&&i.dateFormat!==undefined&&i.minDate===undefined&&(r.settings.minDate=this._formatDate(r,o)),u!==null&&i.dateFormat!==undefined&&i.maxDate===undefined&&(r.settings.maxDate=this._formatDate(r,u)),this._attachments($(e),r),this._autoSize(r),this._setDate(r,s),this._updateAlternate(r),this._updateDatepicker(r)}},_changeDatepicker:function(e,t,n){this._optionDatepicker(e,t,n)},_refreshDatepicker:function(e){var t=this._getInst(e);t&&this._updateDatepicker(t)},_setDateDatepicker:function(e,t){var n=this._getInst(e);n&&(this._setDate(n,t),this._updateDatepicker(n),this._updateAlternate(n))},_getDateDatepicker:function(e,t){var n=this._getInst(e);return n&&!n.inline&&this._setDateFromField(n,t),n?this._getDate(n):null},_doKeyDown:function(e){var t=$.datepicker._getInst(e.target),n=!0,r=t.dpDiv.is(".ui-datepicker-rtl");t._keyEvent=!0;if($.datepicker._datepi
 ckerShowing)switch(e.keyCode){case 9:$.datepicker._hideDatepicker(),n=!1;break;case 13:var i=$("td."+$.datepicker._dayOverClass+":not(."+$.datepicker._currentClass+")",t.dpDiv);i[0]&&$.datepicker._selectDay(e.target,t.selectedMonth,t.selectedYear,i[0]);var s=$.datepicker._get(t,"onSelect");if(s){var o=$.datepicker._formatDate(t);s.apply(t.input?t.input[0]:null,[o,t])}else $.datepicker._hideDatepicker();return!1;case 27:$.datepicker._hideDatepicker();break;case 33:$.datepicker._adjustDate(e.target,e.ctrlKey?-$.datepicker._get(t,"stepBigMonths"):-$.datepicker._get(t,"stepMonths"),"M");break;case 34:$.datepicker._adjustDate(e.target,e.ctrlKey?+$.datepicker._get(t,"stepBigMonths"):+$.datepicker._get(t,"stepMonths"),"M");break;case 35:(e.ctrlKey||e.metaKey)&&$.datepicker._clearDate(e.target),n=e.ctrlKey||e.metaKey;break;case 36:(e.ctrlKey||e.metaKey)&&$.datepicker._gotoToday(e.target),n=e.ctrlKey||e.metaKey;break;case 37:(e.ctrlKey||e.metaKey)&&$.datepicker._adjustDate(e.target,r?1:-1,"D
 "),n=e.ctrlKey||e.metaKey,e.originalEvent.altKey&&$.datepicker._adjustDate(e.target,e.ctrlKey?-$.datepicker._get(t,"stepBigMonths"):-$.datepicker._get(t,"stepMonths"),"M");break;case 38:(e.ctrlKey||e.metaKey)&&$.datepicker._adjustDate(e.target,-7,"D"),n=e.ctrlKey||e.metaKey;break;case 39:(e.ctrlKey||e.metaKey)&&$.datepicker._adjustDate(e.target,r?-1:1,"D"),n=e.ctrlKey||e.metaKey,e.originalEvent.altKey&&$.datepicker._adjustDate(e.target,e.ctrlKey?+$.datepicker._get(t,"stepBigMonths"):+$.datepicker._get(t,"stepMonths"),"M");break;case 40:(e.ctrlKey||e.metaKey)&&$.datepicker._adjustDate(e.target,7,"D"),n=e.ctrlKey||e.metaKey;break;default:n=!1}else e.keyCode==36&&e.ctrlKey?$.datepicker._showDatepicker(this):n=!1;n&&(e.preventDefault(),e.stopPropagation())},_doKeyPress:function(e){var t=$.datepicker._getInst(e.target);if($.datepicker._get(t,"constrainInput")){var n=$.datepicker._possibleChars($.datepicker._get(t,"dateFormat")),r=String.fromCharCode(e.charCode==undefined?e.keyCode:e.char
 Code);return e.ctrlKey||e.metaKey||r<" "||!n||n.indexOf(r)>-1}},_doKeyUp:function(e){var t=$.datepicker._getInst(e.target);if(t.input.val()!=t.lastVal)try{var n=$.datepicker.parseDate($.datepicker._get(t,"dateFormat"),t.input?t.input.val():null,$.datepicker._getFormatConfig(t));n&&($.datepicker._setDateFromField(t),$.datepicker._updateAlternate(t),$.datepicker._updateDatepicker(t))}catch(r){$.datepicker.log(r)}return!0},_showDatepicker:function(e){e=e.target||e,e.nodeName.toLowerCase()!="input"&&(e=$("input",e.parentNode)[0]);if($.datepicker._isDisabledDatepicker(e)||$.datepicker._lastInput==e)return;var t=$.datepicker._getInst(e);$.datepicker._curInst&&$.datepicker._curInst!=t&&($.datepicker._curInst.dpDiv.stop(!0,!0),t&&$.datepicker._datepickerShowing&&$.datepicker._hideDatepicker($.datepicker._curInst.input[0]));var n=$.datepicker._get(t,"beforeShow"),r=n?n.apply(e,[e,t]):{};if(r===!1)return;extendRemove(t.settings,r),t.lastVal=null,$.datepicker._lastInput=e,$.datepicker._setDate
 FromField(t),$.datepicker._inDialog&&(e.value=""),$.datepicker._pos||($.datepicker._pos=$.datepicker._findPos(e),$.datepicker._pos[1]+=e.offsetHeight);var i=!1;$(e).parents().each(function(){return i|=$(this).css("position")=="fixed",!i});var s={left:$.datepicker._pos[0],top:$.datepicker._pos[1]};$.datepicker._pos=null,t.dpDiv.empty(),t.dpDiv.css({position:"absolute",display:"block",top:"-1000px"}),$.datepicker._updateDatepicker(t),s=$.datepicker._checkOffset(t,s,i),t.dpDiv.css({position:$.datepicker._inDialog&&$.blockUI?"static":i?"fixed":"absolute",display:"none",left:s.left+"px",top:s.top+"px"});if(!t.inline){var o=$.datepicker._get(t,"showAnim"),u=$.datepicker._get(t,"duration"),a=function(){var e=t.dpDiv.find("iframe.ui-datepicker-cover");if(!!e.length){var n=$.datepicker._getBorders(t.dpDiv);e.css({left:-n[0],top:-n[1],width:t.dpDiv.outerWidth(),height:t.dpDiv.outerHeight()})}};t.dpDiv.zIndex($(e).zIndex()+1),$.datepicker._datepickerShowing=!0,$.effects&&($.effects.effect[o]||
 $.effects[o])?t.dpDiv.show(o,$.datepicker._get(t,"showOptions"),u,a):t.dpDiv[o||"show"](o?u:null,a),(!o||!u)&&a(),t.input.is(":visible")&&!t.input.is(":disabled")&&t.input.focus(),$.datepicker._curInst=t}},_updateDatepicker:function(e){this.maxRows=4;var t=$.datepicker._getBorders(e.dpDiv);instActive=e,e.dpDiv.empty().append(this._generateHTML(e)),this._attachHandlers(e);var n=e.dpDiv.find("iframe.ui-datepicker-cover");!n.length||n.css({left:-t[0],top:-t[1],width:e.dpDiv.outerWidth(),height:e.dpDiv.outerHeight()}),e.dpDiv.find("."+this._dayOverClass+" a").mouseover();var r=this._getNumberOfMonths(e),i=r[1],s=17;e.dpDiv.removeClass("ui-datepicker-multi-2 ui-datepicker-multi-3 ui-datepicker-multi-4").width(""),i>1&&e.dpDiv.addClass("ui-datepicker-multi-"+i).css("width",s*i+"em"),e.dpDiv[(r[0]!=1||r[1]!=1?"add":"remove")+"Class"]("ui-datepicker-multi"),e.dpDiv[(this._get(e,"isRTL")?"add":"remove")+"Class"]("ui-datepicker-rtl"),e==$.datepicker._curInst&&$.datepicker._datepickerShowing&&
 e.input&&e.input.is(":visible")&&!e.input.is(":disabled")&&e.input[0]!=document.activeElement&&e.input.focus();if(e.yearshtml){var o=e.yearshtml;setTimeout(function(){o===e.yearshtml&&e.yearshtml&&e.dpDiv.find("select.ui-datepicker-year:first").replaceWith(e.yearshtml),o=e.yearshtml=null},0)}},_getBorders:function(e){var t=function(e){return{thin:1,medium:2,thick:3}[e]||e};return[parseFloat(t(e.css("border-left-width"))),parseFloat(t(e.css("border-top-width")))]},_checkOffset:function(e,t,n){var r=e.dpDiv.outerWidth(),i=e.dpDiv.outerHeight(),s=e.input?e.input.outerWidth():0,o=e.input?e.input.outerHeight():0,u=document.documentElement.clientWidth+(n?0:$(document).scrollLeft()),a=document.documentElement.clientHeight+(n?0:$(document).scrollTop());return t.left-=this._get(e,"isRTL")?r-s:0,t.left-=n&&t.left==e.input.offset().left?$(document).scrollLeft():0,t.top-=n&&t.top==e.input.offset().top+o?$(document).scrollTop():0,t.left-=Math.min(t.left,t.left+r>u&&u>r?Math.abs(t.left+r-u):0),t.
 top-=Math.min(t.top,t.top+i>a&&a>i?Math.abs(i+o):0),t},_findPos:function(e){var t=this._getInst(e),n=this._get(t,"isRTL");while(e&&(e.type=="hidden"||e.nodeType!=1||$.expr.filters.hidden(e)))e=e[n?"previousSibling":"nextSibling"];var r=$(e).offset();return[r.left,r.top]},_hideDatepicker:function(e){var t=this._curInst;if(!t||e&&t!=$.data(e,PROP_NAME))return;if(this._datepickerShowing){var n=this._get(t,"showAnim"),r=this._get(t,"duration"),i=function(){$.datepicker._tidyDialog(t)};$.effects&&($.effects.effect[n]||$.effects[n])?t.dpDiv.hide(n,$.datepicker._get(t,"showOptions"),r,i):t.dpDiv[n=="slideDown"?"slideUp":n=="fadeIn"?"fadeOut":"hide"](n?r:null,i),n||i(),this._datepickerShowing=!1;var s=this._get(t,"onClose");s&&s.apply(t.input?t.input[0]:null,[t.input?t.input.val():"",t]),this._lastInput=null,this._inDialog&&(this._dialogInput.css({position:"absolute",left:"0",top:"-100px"}),$.blockUI&&($.unblockUI(),$("body").append(this.dpDiv))),this._inDialog=!1}},_tidyDialog:function(e){
 e.dpDiv.removeClass(this._dialogClass).unbind(".ui-datepicker-calendar")},_checkExternalClick:function(e){if(!$.datepicker._curInst)return;var t=$(e.target),n=$.datepicker._getInst(t[0]);(t[0].id!=$.datepicker._mainDivId&&t.parents("#"+$.datepicker._mainDivId).length==0&&!t.hasClass($.datepicker.markerClassName)&&!t.closest("."+$.datepicker._triggerClass).length&&$.datepicker._datepickerShowing&&(!$.datepicker._inDialog||!$.blockUI)||t.hasClass($.datepicker.markerClassName)&&$.datepicker._curInst!=n)&&$.datepicker._hideDatepicker()},_adjustDate:function(e,t,n){var r=$(e),i=this._getInst(r[0]);if(this._isDisabledDatepicker(r[0]))return;this._adjustInstDate(i,t+(n=="M"?this._get(i,"showCurrentAtPos"):0),n),this._updateDatepicker(i)},_gotoToday:function(e){var t=$(e),n=this._getInst(t[0]);if(this._get(n,"gotoCurrent")&&n.currentDay)n.selectedDay=n.currentDay,n.drawMonth=n.selectedMonth=n.currentMonth,n.drawYear=n.selectedYear=n.currentYear;else{var r=new Date;n.selectedDay=r.getDate(),
 n.drawMonth=n.selectedMonth=r.getMonth(),n.drawYear=n.selectedYear=r.getFullYear()}this._notifyChange(n),this._adjustDate(t)},_selectMonthYear:function(e,t,n){var r=$(e),i=this._getInst(r[0]);i["selected"+(n=="M"?"Month":"Year")]=i["draw"+(n=="M"?"Month":"Year")]=parseInt(t.options[t.selectedIndex].value,10),this._notifyChange(i),this._adjustDate(r)},_selectDay:function(e,t,n,r){var i=$(e);if($(r).hasClass(this._unselectableClass)||this._isDisabledDatepicker(i[0]))return;var s=this._getInst(i[0]);s.selectedDay=s.currentDay=$("a",r).html(),s.selectedMonth=s.currentMonth=t,s.selectedYear=s.currentYear=n,this._selectDate(e,this._formatDate(s,s.currentDay,s.currentMonth,s.currentYear))},_clearDate:function(e){var t=$(e),n=this._getInst(t[0]);this._selectDate(t,"")},_selectDate:function(e,t){var n=$(e),r=this._getInst(n[0]);t=t!=null?t:this._formatDate(r),r.input&&r.input.val(t),this._updateAlternate(r);var i=this._get(r,"onSelect");i?i.apply(r.input?r.input[0]:null,[t,r]):r.input&&r.inp
 ut.trigger("change"),r.inline?this._updateDatepicker(r):(this._hideDatepicker(),this._lastInput=r.input[0],typeof r.input[0]!="object"&&r.input.focus(),this._lastInput=null)},_updateAlternate:function(e){var t=this._get(e,"altField");if(t){var n=this._get(e,"altFormat")||this._get(e,"dateFormat"),r=this._getDate(e),i=this.formatDate(n,r,this._getFormatConfig(e));$(t).each(function(){$(this).val(i)})}},noWeekends:function(e){var t=e.getDay();return[t>0&&t<6,""]},iso8601Week:function(e){var t=new Date(e.getTime());t.setDate(t.getDate()+4-(t.getDay()||7));var n=t.getTime();return t.setMonth(0),t.setDate(1),Math.floor(Math.round((n-t)/864e5)/7)+1},parseDate:function(e,t,n){if(e==null||t==null)throw"Invalid arguments";t=typeof t=="object"?t.toString():t+"";if(t=="")return null;var r=(n?n.shortYearCutoff:null)||this._defaults.shortYearCutoff;r=typeof r!="string"?r:(new Date).getFullYear()%100+parseInt(r,10);var i=(n?n.dayNamesShort:null)||this._defaults.dayNamesShort,s=(n?n.dayNames:null)
 ||this._defaults.dayNames,o=(n?n.monthNamesShort:null)||this._defaults.monthNamesShort,u=(n?n.monthNames:null)||this._defaults.monthNames,a=-1,f=-1,l=-1,c=-1,h=!1,p=function(t){var n=y+1<e.length&&e.charAt(y+1)==t;return n&&y++,n},d=function(e){var n=p(e),r=e=="@"?14:e=="!"?20:e=="y"&&n?4:e=="o"?3:2,i=new RegExp("^\\d{1,"+r+"}"),s=t.substring(g).match(i);if(!s)throw"Missing number at position "+g;return g+=s[0].length,parseInt(s[0],10)},v=function(e,n,r){var i=$.map(p(e)?r:n,function(e,t){return[[t,e]]}).sort(function(e,t){return-(e[1].length-t[1].length)}),s=-1;$.each(i,function(e,n){var r=n[1];if(t.substr(g,r.length).toLowerCase()==r.toLowerCase())return s=n[0],g+=r.length,!1});if(s!=-1)return s+1;throw"Unknown name at position "+g},m=function(){if(t.charAt(g)!=e.charAt(y))throw"Unexpected literal at position "+g;g++},g=0;for(var y=0;y<e.length;y++)if(h)e.charAt(y)=="'"&&!p("'")?h=!1:m();else switch(e.charAt(y)){case"d":l=d("d");break;case"D":v("D",i,s);break;case"o":c=d("o");brea
 k;case"m":f=d("m");break;case"M":f=v("M",o,u);break;case"y":a=d("y");break;case"@":var b=new Date(d("@"));a=b.getFullYear(),f=b.getMonth()+1,l=b.getDate();break;case"!":var b=new Date((d("!")-this._ticksTo1970)/1e4);a=b.getFullYear(),f=b.getMonth()+1,l=b.getDate();break;case"'":p("'")?m():h=!0;break;default:m()}if(g<t.length){var w=t.substr(g);if(!/^\s+/.test(w))throw"Extra/unparsed characters found in date: "+w}a==-1?a=(new Date).getFullYear():a<100&&(a+=(new Date).getFullYear()-(new Date).getFullYear()%100+(a<=r?0:-100));if(c>-1){f=1,l=c;do{var E=this._getDaysInMonth(a,f-1);if(l<=E)break;f++,l-=E}while(!0)}var b=this._daylightSavingAdjust(new Date(a,f-1,l));if(b.getFullYear()!=a||b.getMonth()+1!=f||b.getDate()!=l)throw"Invalid date";return b},ATOM:"yy-mm-dd",COOKIE:"D, dd M yy",ISO_8601:"yy-mm-dd",RFC_822:"D, d M y",RFC_850:"DD, dd-M-y",RFC_1036:"D, d M y",RFC_1123:"D, d M yy",RFC_2822:"D, d M yy",RSS:"D, d M y",TICKS:"!",TIMESTAMP:"@",W3C:"yy-mm-dd",_ticksTo1970:(718685+Math.floo
 r(492.5)-Math.floor(19.7)+Math.floor(4.925))*24*60*60*1e7,formatDate:function(e,t,n){if(!t)return"";var r=(n?n.dayNamesShort:null)||this._defaults.dayNamesShort,i=(n?n.dayNames:null)||this._defaults.dayNames,s=(n?n.monthNamesShort:null)||this._defaults.monthNamesShort,o=(n?n.monthNames:null)||this._defaults.monthNames,u=function(t){var n=h+1<e.length&&e.charAt(h+1)==t;return n&&h++,n},a=function(e,t,n){var r=""+t;if(u(e))while(r.length<n)r="0"+r;return r},f=function(e,t,n,r){return u(e)?r[t]:n[t]},l="",c=!1;if(t)for(var h=0;h<e.length;h++)if(c)e.charAt(h)=="'"&&!u("'")?c=!1:l+=e.charAt(h);else switch(e.charAt(h)){case"d":l+=a("d",t.getDate(),2);break;case"D":l+=f("D",t.getDay(),r,i);break;case"o":l+=a("o",Math.round(((new Date(t.getFullYear(),t.getMonth(),t.getDate())).getTime()-(new Date(t.getFullYear(),0,0)).getTime())/864e5),3);break;case"m":l+=a("m",t.getMonth()+1,2);break;case"M":l+=f("M",t.getMonth(),s,o);break;case"y":l+=u("y")?t.getFullYear():(t.getYear()%100<10?"0":"")+t.ge
 tYear()%100;break;case"@":l+=t.getTime();break;case"!":l+=t.getTime()*1e4+this._ticksTo1970;break;case"'":u("'")?l+="'":c=!0;break;default:l+=e.charAt(h)}return l},_possibleChars:function(e){var t="",n=!1,r=function(t){var n=i+1<e.length&&e.charAt(i+1)==t;return n&&i++,n};for(var i=0;i<e.length;i++)if(n)e.charAt(i)=="'"&&!r("'")?n=!1:t+=e.charAt(i);else switch(e.charAt(i)){case"d":case"m":case"y":case"@":t+="0123456789";break;case"D":case"M":return null;case"'":r("'")?t+="'":n=!0;break;default:t+=e.charAt(i)}return t},_get:function(e,t){return e.settings[t]!==undefined?e.settings[t]:this._defaults[t]},_setDateFromField:function(e,t){if(e.input.val()==e.lastVal)return;var n=this._get(e,"dateFormat"),r=e.lastVal=e.input?e.input.val():null,i,s;i=s=this._getDefaultDate(e);var o=this._getFormatConfig(e);try{i=this.parseDate(n,r,o)||s}catch(u){this.log(u),r=t?"":r}e.selectedDay=i.getDate(),e.drawMonth=e.selectedMonth=i.getMonth(),e.drawYear=e.selectedYear=i.getFullYear(),e.currentDay=r?i.
 getDate():0,e.currentMonth=r?i.getMonth():0,e.currentYear=r?i.getFullYear():0,this._adjustInstDate(e)},_getDefaultDate:function(e){return this._restrictMinMax(e,this._determineDate(e,this._get(e,"defaultDate"),new Date))},_determineDate:function(e,t,n){var r=function(e){var t=new Date;return t.setDate(t.getDate()+e),t},i=function(t){try{return $.datepicker.parseDate($.datepicker._get(e,"dateFormat"),t,$.datepicker._getFormatConfig(e))}catch(n){}var r=(t.toLowerCase().match(/^c/)?$.datepicker._getDate(e):null)||new Date,i=r.getFullYear(),s=r.getMonth(),o=r.getDate(),u=/([+-]?[0-9]+)\s*(d|D|w|W|m|M|y|Y)?/g,a=u.exec(t);while(a){switch(a[2]||"d"){case"d":case"D":o+=parseInt(a[1],10);break;case"w":case"W":o+=parseInt(a[1],10)*7;break;case"m":case"M":s+=parseInt(a[1],10),o=Math.min(o,$.datepicker._getDaysInMonth(i,s));break;case"y":case"Y":i+=parseInt(a[1],10),o=Math.min(o,$.datepicker._getDaysInMonth(i,s))}a=u.exec(t)}return new Date(i,s,o)},s=t==null||t===""?n:typeof t=="string"?i(t):ty
 peof t=="number"?isNaN(t)?n:r(t):new Date(t.getTime());return s=s&&s.toString()=="Invalid Date"?n:s,s&&(s.setHours(0),s.setMinutes(0),s.setSeconds(0),s.setMilliseconds(0)),this._daylightSavingAdjust(s)},_daylightSavingAdjust:function(e){return e?(e.setHours(e.getHours()>12?e.getHours()+2:0),e):null},_setDate:function(e,t,n){var r=!t,i=e.selectedMonth,s=e.selectedYear,o=this._restrictMinMax(e,this._determineDate(e,t,new Date));e.selectedDay=e.currentDay=o.getDate(),e.drawMonth=e.selectedMonth=e.currentMonth=o.getMonth(),e.drawYear=e.selectedYear=e.currentYear=o.getFullYear(),(i!=e.selectedMonth||s!=e.selectedYear)&&!n&&this._notifyChange(e),this._adjustInstDate(e),e.input&&e.input.val(r?"":this._formatDate(e))},_getDate:function(e){var t=!e.currentYear||e.input&&e.input.val()==""?null:this._daylightSavingAdjust(new Date(e.currentYear,e.currentMonth,e.currentDay));return t},_attachHandlers:function(e){var t=this._get(e,"stepMonths"),n="#"+e.id.replace(/\\\\/g,"\\");e.dpDiv.find("[data
 -handler]").map(function(){var e={prev:function(){window["DP_jQuery_"+dpuuid].datepicker._adjustDate(n,-t,"M")},next:function(){window["DP_jQuery_"+dpuuid].datepicker._adjustDate(n,+t,"M")},hide:function(){window["DP_jQuery_"+dpuuid].datepicker._hideDatepicker()},today:function(){window["DP_jQuery_"+dpuuid].datepicker._gotoToday(n)},selectDay:function(){return window["DP_jQuery_"+dpuuid].datepicker._selectDay(n,+this.getAttribute("data-month"),+this.getAttribute("data-year"),this),!1},selectMonth:function(){return window["DP_jQuery_"+dpuuid].datepicker._selectMonthYear(n,this,"M"),!1},selectYear:function(){return window["DP_jQuery_"+dpuuid].datepicker._selectMonthYear(n,this,"Y"),!1}};$(this).bind(this.getAttribute("data-event"),e[this.getAttribute("data-handler")])})},_generateHTML:function(e){var t=new Date;t=this._daylightSavingAdjust(new Date(t.getFullYear(),t.getMonth(),t.getDate()));var n=this._get(e,"isRTL"),r=this._get(e,"showButtonPanel"),i=this._get(e,"hideIfNoPrevNext"),s
 =this._get(e,"navigationAsDateFormat"),o=this._getNumberOfMonths(e),u=this._get(e,"showCurrentAtPos"),a=this._get(e,"stepMonths"),f=o[0]!=1||o[1]!=1,l=this._daylightSavingAdjust(e.currentDay?new Date(e.currentYear,e.currentMonth,e.currentDay):new Date(9999,9,9)),c=this._getMinMaxDate(e,"min"),h=this._getMinMaxDate(e,"max"),p=e.drawMonth-u,d=e.drawYear;p<0&&(p+=12,d--);if(h){var v=this._daylightSavingAdjust(new Date(h.getFullYear(),h.getMonth()-o[0]*o[1]+1,h.getDate()));v=c&&v<c?c:v;while(this._daylightSavingAdjust(new Date(d,p,1))>v)p--,p<0&&(p=11,d--)}e.drawMonth=p,e.drawYear=d;var m=this._get(e,"prevText");m=s?this.formatDate(m,this._daylightSavingAdjust(new Date(d,p-a,1)),this._getFormatConfig(e)):m;var g=this._canAdjustMonth(e,-1,d,p)?'<a class="ui-datepicker-prev ui-corner-all" data-handler="prev" data-event="click" title="'+m+'"><span class="ui-icon ui-icon-circle-triangle-'+(n?"e":"w")+'">'+m+"</span></a>":i?"":'<a class="ui-datepicker-prev ui-corner-all ui-state-disabled" ti
 tle="'+m+'"><span class="ui-icon ui-icon-circle-triangle-'+(n?"e":"w")+'">'+m+"</span></a>",y=this._get(e,"nextText");y=s?this.formatDate(y,this._daylightSavingAdjust(new Date(d,p+a,1)),this._getFormatConfig(e)):y;var b=this._canAdjustMonth(e,1,d,p)?'<a class="ui-datepicker-next ui-corner-all" data-handler="next" data-event="click" title="'+y+'"><span class="ui-icon ui-icon-circle-triangle-'+(n?"w":"e")+'">'+y+"</span></a>":i?"":'<a class="ui-datepicker-next ui-corner-all ui-state-disabled" title="'+y+'"><span class="ui-icon ui-icon-circle-triangle-'+(n?"w":"e")+'">'+y+"</span></a>",w=this._get(e,"currentText"),E=this._get(e,"gotoCurrent")&&e.currentDay?l:t;w=s?this.formatDate(w,E,this._getFormatConfig(e)):w;var S=e.inline?"":'<button type="button" class="ui-datepicker-close ui-state-default ui-priority-primary ui-corner-all" data-handler="hide" data-event="click">'+this._get(e,"closeText")+"</button>",x=r?'<div class="ui-datepicker-buttonpane ui-widget-content">'+(n?S:"")+(this._is
 InRange(e,E)?'<button type="button" class="ui-datepicker-current ui-state-default ui-priority-secondary ui-corner-all" data-handler="today" data-event="click">'+w+"</button>":"")+(n?"":S)+"</div>":"",T=parseInt(this._get(e,"firstDay"),10);T=isNaN(T)?0:T;var N=this._get(e,"showWeek"),C=this._get(e,"dayNames"),k=this._get(e,"dayNamesShort"),L=this._get(e,"dayNamesMin"),A=this._get(e,"monthNames"),O=this._get(e,"monthNamesShort"),M=this._get(e,"beforeShowDay"),_=this._get(e,"showOtherMonths"),D=this._get(e,"selectOtherMonths"),P=this._get(e,"calculateWeek")||this.iso8601Week,H=this._getDefaultDate(e),B="";for(var j=0;j<o[0];j++){var F="";this.maxRows=4;for(var I=0;I<o[1];I++){var q=this._daylightSavingAdjust(new Date(d,p,e.selectedDay)),R=" ui-corner-all",U="";if(f){U+='<div class="ui-datepicker-group';if(o[1]>1)switch(I){case 0:U+=" ui-datepicker-group-first",R=" ui-corner-"+(n?"right":"left");break;case o[1]-1:U+=" ui-datepicker-group-last",R=" ui-corner-"+(n?"left":"right");break;de
 fault:U+=" ui-datepicker-group-middle",R=""}U+='">'}U+='<div class="ui-datepicker-header ui-widget-header ui-helper-clearfix'+R+'">'+(/all|left/.test(R)&&j==0?n?b:g:"")+(/all|right/.test(R)&&j==0?n?g:b:"")+this._generateMonthYearHeader(e,p,d,c,h,j>0||I>0,A,O)+'</div><table class="ui-datepicker-calendar"><thead>'+"<tr>";var z=N?'<th class="ui-datepicker-week-col">'+this._get(e,"weekHeader")+"</th>":"";for(var W=0;W<7;W++){var X=(W+T)%7;z+="<th"+((W+T+6)%7>=5?' class="ui-datepicker-week-end"':"")+">"+'<span title="'+C[X]+'">'+L[X]+"</span></th>"}U+=z+"</tr></thead><tbody>";var V=this._getDaysInMonth(d,p);d==e.selectedYear&&p==e.selectedMonth&&(e.selectedDay=Math.min(e.selectedDay,V));var J=(this._getFirstDayOfMonth(d,p)-T+7)%7,K=Math.ceil((J+V)/7),Q=f?this.maxRows>K?this.maxRows:K:K;this.maxRows=Q;var G=this._daylightSavingAdjust(new Date(d,p,1-J));for(var Y=0;Y<Q;Y++){U+="<tr>";var Z=N?'<td class="ui-datepicker-week-col">'+this._get(e,"calculateWeek")(G)+"</td>":"";for(var W=0;W<7;W+
 +){var et=M?M.apply(e.input?e.input[0]:null,[G]):[!0,""],tt=G.getMonth()!=p,nt=tt&&!D||!et[0]||c&&G<c||h&&G>h;Z+='<td class="'+((W+T+6)%7>=5?" ui-datepicker-week-end":"")+(tt?" ui-datepicker-other-month":"")+(G.getTime()==q.getTime()&&p==e.selectedMonth&&e._keyEvent||H.getTime()==G.getTime()&&H.getTime()==q.getTime()?" "+this._dayOverClass:"")+(nt?" "+this._unselectableClass+" ui-state-disabled":"")+(tt&&!_?"":" "+et[1]+(G.getTime()==l.getTime()?" "+this._currentClass:"")+(G.getTime()==t.getTime()?" ui-datepicker-today":""))+'"'+((!tt||_)&&et[2]?' title="'+et[2]+'"':"")+(nt?"":' data-handler="selectDay" data-event="click" data-month="'+G.getMonth()+'" data-year="'+G.getFullYear()+'"')+">"+(tt&&!_?"&#xa0;":nt?'<span class="ui-state-default">'+G.getDate()+"</span>":'<a class="ui-state-default'+(G.getTime()==t.getTime()?" ui-state-highlight":"")+(G.getTime()==l.getTime()?" ui-state-active":"")+(tt?" ui-priority-secondary":"")+'" href="#">'+G.getDate()+"</a>")+"</td>",G.setDate(G.getDat
 e()+1),G=this._daylightSavingAdjust(G)}U+=Z+"</tr>"}p++,p>11&&(p=0,d++),U+="</tbody></table>"+(f?"</div>"+(o[0]>0&&I==o[1]-1?'<div class="ui-datepicker-row-break"></div>':""):""),F+=U}B+=F}return B+=x+($.ui.ie6&&!e.inline?'<iframe src="javascript:false;" class="ui-datepicker-cover" frameborder="0"></iframe>':""),e._keyEvent=!1,B},_generateMonthYearHeader:function(e,t,n,r,i,s,o,u){var a=this._get(e,"changeMonth"),f=this._get(e,"changeYear"),l=this._get(e,"showMonthAfterYear"),c='<div class="ui-datepicker-title">',h="";if(s||!a)h+='<span class="ui-datepicker-month">'+o[t]+"</span>";else{var p=r&&r.getFullYear()==n,d=i&&i.getFullYear()==n;h+='<select class="ui-datepicker-month" data-handler="selectMonth" data-event="change">';for(var v=0;v<12;v++)(!p||v>=r.getMonth())&&(!d||v<=i.getMonth())&&(h+='<option value="'+v+'"'+(v==t?' selected="selected"':"")+">"+u[v]+"</option>");h+="</select>"}l||(c+=h+(s||!a||!f?"&#xa0;":""));if(!e.yearshtml){e.yearshtml="";if(s||!f)c+='<span class="ui-date
 picker-year">'+n+"</span>";else{var m=this._get(e,"yearRange").split(":"),g=(new Date).getFullYear(),y=function(e){var t=e.match(/c[+-].*/)?n+parseInt(e.substring(1),10):e.match(/[+-].*/)?g+parseInt(e,10):parseInt(e,10);return isNaN(t)?g:t},b=y(m[0]),w=Math.max(b,y(m[1]||""));b=r?Math.max(b,r.getFullYear()):b,w=i?Math.min(w,i.getFullYear()):w,e.yearshtml+='<select class="ui-datepicker-year" data-handler="selectYear" data-event="change">';for(;b<=w;b++)e.yearshtml+='<option value="'+b+'"'+(b==n?' selected="selected"':"")+">"+b+"</option>";e.yearshtml+="</select>",c+=e.yearshtml,e.yearshtml=null}}return c+=this._get(e,"yearSuffix"),l&&(c+=(s||!a||!f?"&#xa0;":"")+h),c+="</div>",c},_adjustInstDate:function(e,t,n){var r=e.drawYear+(n=="Y"?t:0),i=e.drawMonth+(n=="M"?t:0),s=Math.min(e.selectedDay,this._getDaysInMonth(r,i))+(n=="D"?t:0),o=this._restrictMinMax(e,this._daylightSavingAdjust(new Date(r,i,s)));e.selectedDay=o.getDate(),e.drawMonth=e.selectedMonth=o.getMonth(),e.drawYear=e.select
 edYear=o.getFullYear(),(n=="M"||n=="Y")&&this._notifyChange(e)},_restrictMinMax:function(e,t){var n=this._getMinMaxDate(e,"min"),r=this._getMinMaxDate(e,"max"),i=n&&t<n?n:t;return i=r&&i>r?r:i,i},_notifyChange:function(e){var t=this._get(e,"onChangeMonthYear");t&&t.apply(e.input?e.input[0]:null,[e.selectedYear,e.selectedMonth+1,e])},_getNumberOfMonths:function(e){var t=this._get(e,"numberOfMonths");return t==null?[1,1]:typeof t=="number"?[1,t]:t},_getMinMaxDate:function(e,t){return this._determineDate(e,this._get(e,t+"Date"),null)},_getDaysInMonth:function(e,t){return 32-this._daylightSavingAdjust(new Date(e,t,32)).getDate()},_getFirstDayOfMonth:function(e,t){return(new Date(e,t,1)).getDay()},_canAdjustMonth:function(e,t,n,r){var i=this._getNumberOfMonths(e),s=this._daylightSavingAdjust(new Date(n,r+(t<0?t:i[0]*i[1]),1));return t<0&&s.setDate(this._getDaysInMonth(s.getFullYear(),s.getMonth())),this._isInRange(e,s)},_isInRange:function(e,t){var n=this._getMinMaxDate(e,"min"),r=this._
 getMinMaxDate(e,"max");return(!n||t.getTime()>=n.getTime())&&(!r||t.getTime()<=r.getTime())},_getFormatConfig:function(e){var t=this._get(e,"shortYearCutoff");return t=typeof t!="string"?t:(new Date).getFullYear()%100+parseInt(t,10),{shortYearCutoff:t,dayNamesShort:this._get(e,"dayNamesShort"),dayNames:this._get(e,"dayNames"),monthNamesShort:this._get(e,"monthNamesShort"),monthNames:this._get(e,"monthNames")}},_formatDate:function(e,t,n,r){t||(e.currentDay=e.selectedDay,e.currentMonth=e.selectedMonth,e.currentYear=e.selectedYear);var i=t?typeof t=="object"?t:this._daylightSavingAdjust(new Date(r,n,t)):this._daylightSavingAdjust(new Date(e.currentYear,e.currentMonth,e.currentDay));return this.formatDate(this._get(e,"dateFormat"),i,this._getFormatConfig(e))}}),$.fn.datepicker=function(e){if(!this.length)return this;$.datepicker.initialized||($(document).mousedown($.datepicker._checkExternalClick).find(document.body).append($.datepicker.dpDiv),$.datepicker.initialized=!0);var t=Array.p
 rototype.slice.call(arguments,1);return typeof e!="string"||e!="isDisabled"&&e!="getDate"&&e!="widget"?e=="option"&&arguments.length==2&&typeof arguments[1]=="string"?$.datepicker["_"+e+"Datepicker"].apply($.datepicker,[this[0]].concat(t)):this.each(function(){typeof e=="string"?$.datepicker["_"+e+"Datepicker"].apply($.datepicker,[this].concat(t)):$.datepicker._attachDatepicker(this,e)}):$.datepicker["_"+e+"Datepicker"].apply($.datepicker,[this[0]].concat(t))},$.datepicker=new Datepicker,$.datepicker.initialized=!1,$.datepicker.uuid=(new Date).getTime(),$.datepicker.version="1.9.1",window["DP_jQuery_"+dpuuid]=$})(jQuery);(function(e,t){var n="ui-dialog ui-widget ui-widget-content ui-corner-all ",r={buttons:!0,height:!0,maxHeight:!0,maxWidth:!0,minHeight:!0,minWidth:!0,width:!0},i={maxHeight:!0,maxWidth:!0,minHeight:!0,minWidth:!0};e.widget("ui.dialog",{version:"1.9.1",options:{autoOpen:!0,buttons:{},closeOnEscape:!0,closeText:"close",dialogClass:"",draggable:!0,hide:null,height:"aut
 o",maxHeight:!1,maxWidth:!1,minHeight:150,minWidth:150,modal:!1,position:{my:"center",at:"center",of:window,collision:"fit",using:function(t){var n=e(this).css(t).offset().top;n<0&&e(this).css("top",t.top-n)}},resizable:!0,show:null,stack:!0,title:"",width:300,zIndex:1e3},_create:function(){this.originalTitle=this.element.attr("title"),typeof this.originalTitle!="string"&&(this.originalTitle=""),this.oldPosition={parent:this.element.parent(),index:this.element.parent().children().index(this.element)},this.options.title=this.options.title||this.originalTitle;var t=this,r=this.options,i=r.title||"&#160;",s,o,u,a,f;s=(this.uiDialog=e("<div>")).addClass(n+r.dialogClass).css({display:"none",outline:0,zIndex:r.zIndex}).attr("tabIndex",-1).keydown(function(n){r.closeOnEscape&&!n.isDefaultPrevented()&&n.keyCode&&n.keyCode===e.ui.keyCode.ESCAPE&&(t.close(n),n.preventDefault())}).mousedown(function(e){t.moveToTop(!1,e)}).appendTo("body"),this.element.show().removeAttr("title").addClass("ui-di
 alog-content ui-widget-content").appendTo(s),o=(this.uiDialogTitlebar=e("<div>")).addClass("ui-dialog-titlebar  ui-widget-header  ui-corner-all  ui-helper-clearfix").bind("mousedown",function(){s.focus()}).prependTo(s),u=e("<a href='#'></a>").addClass("ui-dialog-titlebar-close  ui-corner-all").attr("role","button").click(function(e){e.preventDefault(),t.close(e)}).appendTo(o),(this.uiDialogTitlebarCloseText=e("<span>")).addClass("ui-icon ui-icon-closethick").text(r.closeText).appendTo(u),a=e("<span>").uniqueId().addClass("ui-dialog-title").html(i).prependTo(o),f=(this.uiDialogButtonPane=e("<div>")).addClass("ui-dialog-buttonpane ui-widget-content ui-helper-clearfix"),(this.uiButtonSet=e("<div>")).addClass("ui-dialog-buttonset").appendTo(f),s.attr({role:"dialog","aria-labelledby":a.attr("id")}),o.find("*").add(o).disableSelection(),this._hoverable(u),this._focusable(u),r.draggable&&e.fn.draggable&&this._makeDraggable(),r.resizable&&e.fn.resizable&&this._makeResizable(),this._createBu
 ttons(r.buttons),this._isOpen=!1,e.fn.bgiframe&&s.bgiframe(),this._on(s,{keydown:function(t){if(!r.modal||t.keyCode!==e.ui.keyCode.TAB)return;var n=e(":tabbable",s),i=n.filter(":first"),o=n.filter(":last");if(t.target===o[0]&&!t.shiftKey)return i.focus(1),!1;if(t.target===i[0]&&t.shiftKey)return o.focus(1),!1}})},_init:function(){this.options.autoOpen&&this.open()},_destroy:function(){var e,t=this.oldPosition;this.overlay&&this.overlay.destroy(),this.uiDialog.hide(),this.element.removeClass("ui-dialog-content ui-widget-content").hide().appendTo("body"),this.uiDialog.remove(),this.originalTitle&&this.element.attr("title",this.originalTitle),e=t.parent.children().eq(t.index),e.length&&e[0]!==this.element[0]?e.before(this.element):t.parent.append(this.element)},widget:function(){return this.uiDialog},close:function(t){var n=this,r,i;if(!this._isOpen)return;if(!1===this._trigger("beforeClose",t))return;return this._isOpen=!1,this.overlay&&this.overlay.destroy(),this.options.hide?this._h
 ide(this.uiDialog,this.options.hide,function(){n._trigger("close",t)}):(this.uiDialog.hide(),this._trigger("close",t)),e.ui.dialog.overlay.resize(),this.options.modal&&(r=0,e(".ui-dialog").each(function(){this!==n.uiDialog[0]&&(i=e(this).css("z-index"),isNaN(i)||(r=Math.max(r,i)))}),e.ui.dialog.maxZ=r),this},isOpen:function(){return this._isOpen},moveToTop:function(t,n){var r=this.options,i;return r.modal&&!t||!r.stack&&!r.modal?this._trigger("focus",n):(r.zIndex>e.ui.dialog.maxZ&&(e.ui.dialog.maxZ=r.zIndex),this.overlay&&(e.ui.dialog.maxZ+=1,e.ui.dialog.overlay.maxZ=e.ui.dialog.maxZ,this.overlay.$el.css("z-index",e.ui.dialog.overlay.maxZ)),i={scrollTop:this.element.scrollTop(),scrollLeft:this.element.scrollLeft()},e.ui.dialog.maxZ+=1,this.uiDialog.css("z-index",e.ui.dialog.maxZ),this.element.attr(i),this._trigger("focus",n),this)},open:function(){if(this._isOpen)return;var t,n=this.options,r=this.uiDialog;return this._size(),this._position(n.position),r.show(n.show),this.overlay=n.
 modal?new e.ui.dialog.overlay(this):null,this.moveToTop(!0),t=this.element.find(":tabbable"),t.length||(t=this.uiDialogButtonPane.find(":tabbable"),t.length||(t=r)),t.eq(0).focus(),this._isOpen=!0,this._trigger("open"),this},_createButtons:function(t){var n=this,r=!1;this.uiDialogButtonPane.remove(),this.uiButtonSet.empty(),typeof t=="object"&&t!==null&&e.each(t,function(){return!(r=!0)}),r?(e.each(t,function(t,r){r=e.isFunction(r)?{click:r,text:t}:r;var i=e("<button type='button'></button>").attr(r,!0).unbind("click").click(function(){r.click.apply(n.element[0],arguments)}).appendTo(n.uiButtonSet);e.fn.button&&i.button()}),this.uiDialog.addClass("ui-dialog-buttons"),this.uiDialogButtonPane.appendTo(this.uiDialog)):this.uiDialog.removeClass("ui-dialog-buttons")},_makeDraggable:function(){function r(e){return{position:e.position,offset:e.offset}}var t=this,n=this.options;this.uiDialog.draggable({cancel:".ui-dialog-content, .ui-dialog-titlebar-close",handle:".ui-dialog-titlebar",conta
 inment:"document",start:function(n,i){e(this).addClass

<TRUNCATED>

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


[07/50] [abbrv] hadoop git commit: YARN-5308. FairScheduler: Move continuous scheduling related tests to TestContinuousScheduling (Kai Sasaki via Varun Saxena)

Posted by vi...@apache.org.
YARN-5308. FairScheduler: Move continuous scheduling related tests to TestContinuousScheduling (Kai Sasaki via Varun Saxena)


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

Branch: refs/heads/HDFS-9806
Commit: 79aeddc88f0e71f0031d5f39fded172de0b29a2e
Parents: ac35ee9
Author: Varun Saxena <va...@apache.org>
Authored: Fri Oct 28 00:29:53 2016 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Oct 28 00:34:50 2016 +0530

----------------------------------------------------------------------
 .../fair/TestContinuousScheduling.java          | 189 ++++++++++++++++++-
 .../scheduler/fair/TestFairScheduler.java       | 157 ---------------
 2 files changed, 187 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79aeddc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
index 6188246..5964d2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
@@ -22,20 +22,32 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
 import org.junit.Before;
 import org.junit.Test;
 
@@ -43,18 +55,22 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 public class TestContinuousScheduling extends FairSchedulerTestBase {
   private ControlledClock mockClock;
+  private static int delayThresholdTimeMs = 1000;
 
   @Override
   public Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     conf.setBoolean(
         FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED, true);
-    conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_NODE_MS, 100);
-    conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_RACK_MS, 100);
+    conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_NODE_MS,
+        delayThresholdTimeMs);
+    conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_RACK_MS,
+        delayThresholdTimeMs);
     return conf;
   }
 
@@ -167,6 +183,175 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     Assert.assertEquals(2, nodes.size());
   }
 
+  @Test
+  public void testWithNodeRemoved() throws Exception {
+    // Disable continuous scheduling, will invoke continuous
+    // scheduling once manually
+    scheduler = new FairScheduler();
+    conf = super.createConfiguration();
+    resourceManager = new MockRM(conf);
+
+    // TODO: This test should really be using MockRM. For now starting stuff
+    // that is needed at a bare minimum.
+    ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
+    resourceManager.getRMContext().getStateStore().start();
+
+    // to initialize the master key
+    resourceManager.getRMContext().getContainerTokenSecretManager()
+        .rollMasterKey();
+
+    scheduler.setRMContext(resourceManager.getRMContext());
+    Assert.assertTrue("Continuous scheduling should be disabled.",
+        !scheduler.isContinuousSchedulingEnabled());
+    scheduler.init(conf);
+    scheduler.start();
+
+    // Add two nodes
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+    RMNode node2 =
+        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 2,
+            "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+    Assert.assertEquals("We should have two alive nodes.",
+        2, scheduler.getNumClusterNodes());
+
+    // Remove one node
+    NodeRemovedSchedulerEvent removeNode1
+        = new NodeRemovedSchedulerEvent(node1);
+    scheduler.handle(removeNode1);
+    Assert.assertEquals("We should only have one alive node.",
+        1, scheduler.getNumClusterNodes());
+
+    // Invoke the continuous scheduling once
+    try {
+      scheduler.continuousSchedulingAttempt();
+    } catch (Exception e) {
+      fail("Exception happened when doing continuous scheduling. " +
+          e.toString());
+    }
+  }
+
+  @Test
+  public void testInterruptedException()
+          throws Exception {
+    // Disable continuous scheduling, will invoke continuous
+    // scheduling once manually
+    scheduler = new FairScheduler();
+    conf = super.createConfiguration();
+    resourceManager = new MockRM(conf);
+
+    // TODO: This test should really be using MockRM. For now starting stuff
+    // that is needed at a bare minimum.
+    ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
+    resourceManager.getRMContext().getStateStore().start();
+
+    // to initialize the master key
+    resourceManager.getRMContext().getContainerTokenSecretManager()
+        .rollMasterKey();
+
+    scheduler.setRMContext(resourceManager.getRMContext());
+    scheduler.init(conf);
+    scheduler.start();
+    FairScheduler spyScheduler = spy(scheduler);
+    Assert.assertTrue("Continuous scheduling should be disabled.",
+        !spyScheduler.isContinuousSchedulingEnabled());
+    // Add one nodes
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    spyScheduler.handle(nodeEvent1);
+    Assert.assertEquals("We should have one alive node.",
+        1, spyScheduler.getNumClusterNodes());
+    InterruptedException ie = new InterruptedException();
+    doThrow(new YarnRuntimeException(ie)).when(spyScheduler).
+        attemptScheduling(isA(FSSchedulerNode.class));
+    // Invoke the continuous scheduling once
+    try {
+      spyScheduler.continuousSchedulingAttempt();
+      fail("Expected InterruptedException to stop schedulingThread");
+    } catch (InterruptedException e) {
+      Assert.assertEquals(ie, e);
+    }
+  }
+
+  @Test
+  public void testThreadLifeCycle() throws InterruptedException {
+    scheduler.start();
+
+    Thread updateThread = scheduler.updateThread;
+    Thread schedulingThread = scheduler.schedulingThread;
+
+    assertTrue(updateThread.isAlive());
+    assertTrue(schedulingThread.isAlive());
+
+    scheduler.stop();
+
+    int numRetries = 100;
+    while (numRetries-- > 0 &&
+        (updateThread.isAlive() || schedulingThread.isAlive())) {
+      Thread.sleep(50);
+    }
+
+    assertNotEquals("One of the threads is still alive", 0, numRetries);
+  }
+
+  @Test
+  public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
+    scheduler.start();
+
+    int priorityValue;
+    Priority priority;
+    FSAppAttempt fsAppAttempt;
+    ResourceRequest request1;
+    ResourceRequest request2;
+    ApplicationAttemptId id11;
+
+    priorityValue = 1;
+    id11 = createAppAttemptId(1, 1);
+    createMockRMApp(id11);
+    priority = Priority.newInstance(priorityValue);
+    scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1",
+        false);
+    scheduler.addApplicationAttempt(id11, false, false);
+    fsAppAttempt = scheduler.getApplicationAttempt(id11);
+
+    String hostName = "127.0.0.1";
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(16 * 1024, 16), 1,
+        hostName);
+    List<ResourceRequest> ask1 = new ArrayList<>();
+    request1 =
+        createResourceRequest(1024, 8, node1.getRackName(), priorityValue, 1,
+        true);
+    request2 =
+        createResourceRequest(1024, 8, ResourceRequest.ANY, priorityValue, 1,
+        true);
+    ask1.add(request1);
+    ask1.add(request2);
+    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
+        null, null);
+
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+    FSSchedulerNode node =
+        (FSSchedulerNode) scheduler.getSchedulerNode(node1.getNodeID());
+    // Tick the time and let the fsApp startTime different from initScheduler
+    // time
+    mockClock.tickSec(delayThresholdTimeMs / 1000);
+    scheduler.attemptScheduling(node);
+    Map<SchedulerRequestKey, Long> lastScheduledContainer =
+        fsAppAttempt.getLastScheduledContainer();
+    long initSchedulerTime =
+        lastScheduledContainer.get(TestUtils.toSchedulerKey(priority));
+    assertEquals(delayThresholdTimeMs, initSchedulerTime);
+  }
+
   private void triggerSchedulingAttempt() throws InterruptedException {
     Thread.sleep(
         2 * scheduler.getConf().getContinuousSchedulingSleepMs());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79aeddc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 9e0dd06..e28b35a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -25,10 +25,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
 import java.io.File;
@@ -62,7 +59,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -99,12 +95,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnSched
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-    .TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
@@ -4125,71 +4118,6 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   }
 
   @Test
-  public void testContinuousSchedulingWithNodeRemoved() throws Exception {
-    // Disable continuous scheduling, will invoke continuous scheduling once manually
-    scheduler.init(conf);
-    scheduler.start();
-    Assert.assertTrue("Continuous scheduling should be disabled.",
-        !scheduler.isContinuousSchedulingEnabled());
-
-    // Add two nodes
-    RMNode node1 =
-        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
-            "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent1);
-    RMNode node2 =
-        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 2,
-            "127.0.0.2");
-    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
-    scheduler.handle(nodeEvent2);
-    Assert.assertEquals("We should have two alive nodes.",
-        2, scheduler.getNumClusterNodes());
-
-    // Remove one node
-    NodeRemovedSchedulerEvent removeNode1 = new NodeRemovedSchedulerEvent(node1);
-    scheduler.handle(removeNode1);
-    Assert.assertEquals("We should only have one alive node.",
-        1, scheduler.getNumClusterNodes());
-
-    // Invoke the continuous scheduling once
-    try {
-      scheduler.continuousSchedulingAttempt();
-    } catch (Exception e) {
-      fail("Exception happened when doing continuous scheduling. " +
-        e.toString());
-    }
-  }
-
-  @Test
-  public void testContinuousSchedulingInterruptedException()
-      throws Exception {
-    scheduler.init(conf);
-    scheduler.start();
-    FairScheduler spyScheduler = spy(scheduler);
-    Assert.assertTrue("Continuous scheduling should be disabled.",
-        !spyScheduler.isContinuousSchedulingEnabled());
-    // Add one nodes
-    RMNode node1 =
-        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
-            "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    spyScheduler.handle(nodeEvent1);
-    Assert.assertEquals("We should have one alive node.",
-        1, spyScheduler.getNumClusterNodes());
-    InterruptedException ie = new InterruptedException();
-    doThrow(new YarnRuntimeException(ie)).when(spyScheduler).
-        attemptScheduling(isA(FSSchedulerNode.class));
-    // Invoke the continuous scheduling once
-    try {
-      spyScheduler.continuousSchedulingAttempt();
-      fail("Expected InterruptedException to stop schedulingThread");
-    } catch (InterruptedException e) {
-      Assert.assertEquals(ie, e);
-    }
-  }
-
-  @Test
   public void testSchedulingOnRemovedNode() throws Exception {
     // Disable continuous scheduling, will invoke continuous scheduling manually
     scheduler.init(conf);
@@ -4487,30 +4415,6 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   }
 
   @Test
-  public void testThreadLifeCycle() throws InterruptedException {
-    conf.setBoolean(
-        FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED, true);
-    scheduler.init(conf);
-    scheduler.start();
-
-    Thread updateThread = scheduler.updateThread;
-    Thread schedulingThread = scheduler.schedulingThread;
-
-    assertTrue(updateThread.isAlive());
-    assertTrue(schedulingThread.isAlive());
-
-    scheduler.stop();
-
-    int numRetries = 100;
-    while (numRetries-- > 0 &&
-        (updateThread.isAlive() || schedulingThread.isAlive())) {
-      Thread.sleep(50);
-    }
-
-    assertNotEquals("One of the threads is still alive", 0, numRetries);
-  }
-
-  @Test
   public void testPerfMetricsInited() {
     scheduler.init(conf);
     scheduler.start();
@@ -4645,67 +4549,6 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   }
 
   @Test
-  public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
-    int DELAY_THRESHOLD_TIME_MS = 1000;
-    conf.set(FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED, "true");
-    conf.set(FairSchedulerConfiguration.LOCALITY_DELAY_NODE_MS,
-        String.valueOf(DELAY_THRESHOLD_TIME_MS));
-    conf.set(FairSchedulerConfiguration.LOCALITY_DELAY_RACK_MS,
-        String.valueOf(DELAY_THRESHOLD_TIME_MS));
-
-    ControlledClock clock = new ControlledClock();
-    scheduler.setClock(clock);
-    scheduler.init(conf);
-    scheduler.start();
-
-    int priorityValue;
-    Priority priority;
-    FSAppAttempt fsAppAttempt;
-    ResourceRequest request1;
-    ResourceRequest request2;
-    ApplicationAttemptId id11;
-
-    priorityValue = 1;
-    id11 = createAppAttemptId(1, 1);
-    createMockRMApp(id11);
-    priority = Priority.newInstance(priorityValue);
-    scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1",
-        false);
-    scheduler.addApplicationAttempt(id11, false, false);
-    fsAppAttempt = scheduler.getApplicationAttempt(id11);
-
-    String hostName = "127.0.0.1";
-    RMNode node1 =
-        MockNodes.newNodeInfo(1, Resources.createResource(16 * 1024, 16), 1,
-            hostName);
-    List<ResourceRequest> ask1 = new ArrayList<>();
-    request1 =
-        createResourceRequest(1024, 8, node1.getRackName(), priorityValue, 1,
-            true);
-    request2 =
-        createResourceRequest(1024, 8, ResourceRequest.ANY, priorityValue, 1,
-            true);
-    ask1.add(request1);
-    ask1.add(request2);
-    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
-        null, null);
-
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent1);
-    FSSchedulerNode node =
-        (FSSchedulerNode) scheduler.getSchedulerNode(node1.getNodeID());
-    // Tick the time and let the fsApp startTime different from initScheduler
-    // time
-    clock.tickSec(DELAY_THRESHOLD_TIME_MS / 1000);
-    scheduler.attemptScheduling(node);
-    Map<SchedulerRequestKey, Long> lastScheduledContainer =
-        fsAppAttempt.getLastScheduledContainer();
-    long initSchedulerTime =
-        lastScheduledContainer.get(TestUtils.toSchedulerKey(priority));
-    assertEquals(DELAY_THRESHOLD_TIME_MS, initSchedulerTime);
-  }
-
-  @Test
   public void testResourceUpdateDecommissioningNode() throws Exception {
     // Mock the RMNodeResourceUpdate event handler to update SchedulerNode
     // to have 0 available resource


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


[48/50] [abbrv] hadoop git commit: YARN-5027. NM should clean up app log dirs after NM restart. Contributed by sandflee

Posted by vi...@apache.org.
YARN-5027. NM should clean up app log dirs after NM restart. Contributed by sandflee


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

Branch: refs/heads/HDFS-9806
Commit: 7146359bfd436a76585fb1f3ea93716795308cec
Parents: c017171
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Oct 28 15:48:58 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Oct 28 15:48:58 2016 +0000

----------------------------------------------------------------------
 .../localizer/ResourceLocalizationService.java  | 56 +++++++++++++++++++-
 1 file changed, 54 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7146359b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 2cf6ee9..4bd004b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -242,6 +242,7 @@ public class ResourceLocalizationService extends CompositeService
 
       if (!stateStore.canRecover()|| stateStore.isNewlyCreated()) {
         cleanUpLocalDirs(lfs, delService);
+        cleanupLogDirs(lfs, delService);
         initializeLocalDirs(lfs);
         initializeLogDirs(lfs);
       }
@@ -1374,9 +1375,9 @@ public class ResourceLocalizationService extends CompositeService
     }
   }
 
-  private void initializeLogDir(FileContext lfs, String logDir) {
+  private void initializeLogDir(FileContext fs, String logDir) {
     try {
-      lfs.mkdir(new Path(logDir), null, true);
+      fs.mkdir(new Path(logDir), null, true);
     } catch (FileAlreadyExistsException fe) {
       // do nothing
     } catch (IOException e) {
@@ -1386,6 +1387,57 @@ public class ResourceLocalizationService extends CompositeService
     }
   }
 
+  private void cleanupLogDirs(FileContext fs, DeletionService del) {
+    for (String logDir : dirsHandler.getLogDirsForCleanup()) {
+      try {
+        cleanupLogDir(fs, del, logDir);
+      } catch (IOException e) {
+        LOG.warn("failed to cleanup app log dir " + logDir, e);
+      }
+    }
+  }
+
+  private void cleanupLogDir(FileContext fs, DeletionService del,
+      String logDir) throws IOException {
+    if (!fs.util().exists(new Path(logDir))){
+      return;
+    }
+    renameAppLogDir(logDir);
+    deleteAppLogDir(fs, del, logDir);
+  }
+
+  private void renameAppLogDir(String logDir) throws IOException {
+    long currentTimeStamp = System.currentTimeMillis();
+    RemoteIterator<FileStatus> fileStatuses =
+        lfs.listStatus(new Path(logDir));
+    if (fileStatuses != null) {
+      while (fileStatuses.hasNext()) {
+        FileStatus fileStatus = fileStatuses.next();
+        String appName = fileStatus.getPath().getName();
+        if (appName.matches("^application_\\d+_\\d+$")) {
+          lfs.rename(new Path(logDir, appName),
+              new Path(logDir, appName + "_DEL_" + currentTimeStamp));
+        }
+      }
+    }
+  }
+
+  private void deleteAppLogDir(FileContext fs, DeletionService del,
+      String logDir) throws IOException {
+    RemoteIterator<FileStatus> fileStatuses =
+        fs.listStatus(new Path(logDir));
+    if (fileStatuses != null) {
+      while (fileStatuses.hasNext()) {
+        FileStatus fileStatus = fileStatuses.next();
+        String appName = fileStatus.getPath().getName();
+        if (appName.matches("^application_\\d+_\\d+_DEL_\\d+$")) {
+          LOG.info("delete app log dir," + appName);
+          del.delete(null, fileStatus.getPath());
+        }
+      }
+    }
+  }
+
   private void cleanUpLocalDirs(FileContext lfs, DeletionService del) {
     for (String localDir : dirsHandler.getLocalDirsForCleanup()) {
       cleanUpLocalDir(lfs, del, localDir);


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


[02/50] [abbrv] hadoop git commit: YARN-5420. Delete org.apache.hadoop.yarn.server.resourcemanager.resource.Priority as its not necessary. Contributed by Sunil G.

Posted by vi...@apache.org.
YARN-5420. Delete org.apache.hadoop.yarn.server.resourcemanager.resource.Priority as its not necessary. Contributed by Sunil G.


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

Branch: refs/heads/HDFS-9806
Commit: b3c15e4ef763ebc4b033c686114fe627350824ac
Parents: 060558c
Author: Naganarasimha <na...@apache.org>
Authored: Thu Oct 27 18:22:07 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Thu Oct 27 18:22:07 2016 +0530

----------------------------------------------------------------------
 .../resourcemanager/resource/Priority.java      | 31 ---------------
 .../resourcemanager/TestResourceManager.java    |  6 +--
 ...estProportionalCapacityPreemptionPolicy.java |  4 +-
 .../scheduler/TestSchedulerHealth.java          | 16 ++------
 .../capacity/TestCapacityScheduler.java         | 42 +++++---------------
 .../scheduler/fifo/TestFifoScheduler.java       | 12 ++----
 6 files changed, 23 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3c15e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java
deleted file mode 100644
index f098806..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java
+++ /dev/null
@@ -1,31 +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.yarn.server.resourcemanager.resource;
-
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-
-public class Priority {
-  
-  public static org.apache.hadoop.yarn.api.records.Priority create(int prio) {
-    org.apache.hadoop.yarn.api.records.Priority priority = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(org.apache.hadoop.yarn.api.records.Priority.class);
-    priority.setPriority(prio);
-    return priority;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3c15e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.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/TestResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
index 3b59417..ad8c335 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
@@ -117,8 +117,7 @@ public class TestResourceManager {
     // Application resource requirements
     final int memory1 = 1024;
     Resource capability1 = Resources.createResource(memory1, 1);
-    Priority priority1 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
+    Priority priority1 = Priority.newInstance(1);
     application.addResourceRequestSpec(priority1, capability1);
     
     Task t1 = new Task(application, priority1, new String[] {host1, host2});
@@ -126,8 +125,7 @@ public class TestResourceManager {
     
     final int memory2 = 2048;
     Resource capability2 = Resources.createResource(memory2, 1);
-    Priority priority0 = 
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); // higher
+    Priority priority0 = Priority.newInstance(0); // higher
     application.addResourceRequestSpec(priority0, capability2);
     
     // Send resource requests to the scheduler

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3c15e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 21f6eb5..0f5d526 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -32,7 +33,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
@@ -1318,7 +1318,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     ContainerId cId = ContainerId.newContainerId(appAttId, id);
     Container c = mock(Container.class);
     when(c.getResource()).thenReturn(r);
-    when(c.getPriority()).thenReturn(Priority.create(cpriority));
+    when(c.getPriority()).thenReturn(Priority.newInstance(cpriority));
     SchedulerRequestKey sk = SchedulerRequestKey.extractFrom(c);
     RMContainer mC = mock(RMContainer.class);
     when(mC.getContainerId()).thenReturn(cId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3c15e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java
index df35485..56d38fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java
@@ -207,12 +207,8 @@ public class TestSchedulerHealth {
           Resources.createResource(5 * 1024, 1));
 
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-          .create(0);
-    Priority priority_1 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-          .create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
 
     // Submit an application
     Application application_0 =
@@ -290,12 +286,8 @@ public class TestSchedulerHealth {
     nodeUpdate(nm_1);
 
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-          .create(0);
-    Priority priority_1 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-          .create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
 
     // Submit an application
     Application application_0 =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3c15e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index bc92c01..6bcf949 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -283,10 +283,8 @@ public class TestCapacityScheduler {
           Resources.createResource(2 * GB, 1));
 
     // ResourceRequest priorities
-    Priority priority_0 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); 
-    Priority priority_1 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
     
     // Submit an application
     Application application_0 = new Application("user_0", "a1", resourceManager);
@@ -1514,12 +1512,8 @@ public class TestCapacityScheduler {
             Resources.createResource(2 * GB, 1));
 
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(0);
-    Priority priority_1 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
 
     // Submit application_0
     Application application_0 =
@@ -1634,12 +1628,8 @@ public class TestCapacityScheduler {
             Resources.createResource(5 * GB, 1));
 
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(0);
-    Priority priority_1 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
 
     // Submit application_0
     Application application_0 =
@@ -1754,12 +1744,8 @@ public class TestCapacityScheduler {
             Resources.createResource(6 * GB, 1));
 
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(0);
-    Priority priority_1 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
 
     // Submit application_0
     Application application_0 =
@@ -1812,12 +1798,8 @@ public class TestCapacityScheduler {
             Resources.createResource(5 * GB, 1));
 
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(0);
-    Priority priority_1 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(1);
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
 
     // Submit application_0
     Application application_0 =
@@ -3385,9 +3367,7 @@ public class TestCapacityScheduler {
         registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
             Resources.createResource(8 * GB, 4));
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(0);
+    Priority priority_0 = Priority.newInstance(0);
 
     // Submit an application
     Application application_0 =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3c15e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 84217c4..028bcb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -409,11 +409,9 @@ public class TestFifoScheduler {
     nm_1.heartbeat();
 
     // ResourceRequest priorities
-    Priority priority_0 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); 
-    Priority priority_1 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
-    
+    Priority priority_0 = Priority.newInstance(0);
+    Priority priority_1 = Priority.newInstance(1);
+
     // Submit an application
     Application application_0 = new Application("user_0", resourceManager);
     application_0.submit();
@@ -1205,9 +1203,7 @@ public class TestFifoScheduler {
         registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
             Resources.createResource(8 * GB, 4));
     // ResourceRequest priorities
-    Priority priority_0 =
-        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
-            .create(0);
+    Priority priority_0 = Priority.newInstance(0);
 
     // Submit an application
     Application application_0 =


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


[19/50] [abbrv] hadoop git commit: YARN-4668. Reuse objectMapper instance in Yarn. (Yiqun Lin via gtcarrera9)

Posted by vi...@apache.org.
YARN-4668. Reuse objectMapper instance in Yarn. (Yiqun Lin via gtcarrera9)


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

Branch: refs/heads/HDFS-9806
Commit: 221582c4ab0ff1d5936f754f23da140aac656654
Parents: 892a834
Author: Li Lu <gt...@apache.org>
Authored: Thu Oct 27 15:19:59 2016 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Thu Oct 27 15:20:17 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/TimelineClientImpl.java      | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/221582c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index dc4d3e6..d969c59 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -98,6 +98,7 @@ import com.sun.jersey.core.util.MultivaluedMapImpl;
 public class TimelineClientImpl extends TimelineClient {
 
   private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
   private static final String RESOURCE_URI_STR_V1 = "/ws/v1/timeline/";
   private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/";
   private static final Joiner JOINER = Joiner.on("");
@@ -765,15 +766,14 @@ public class TimelineClientImpl extends TimelineClient {
       LOG.error("File [" + jsonFile.getAbsolutePath() + "] doesn't exist");
       return;
     }
-    ObjectMapper mapper = new ObjectMapper();
-    YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
+    YarnJacksonJaxbJsonProvider.configObjectMapper(MAPPER);
     TimelineEntities entities = null;
     TimelineDomains domains = null;
     try {
       if (type.equals(ENTITY_DATA_TYPE)) {
-        entities = mapper.readValue(jsonFile, TimelineEntities.class);
+        entities = MAPPER.readValue(jsonFile, TimelineEntities.class);
       } else if (type.equals(DOMAIN_DATA_TYPE)){
-        domains = mapper.readValue(jsonFile, TimelineDomains.class);
+        domains = MAPPER.readValue(jsonFile, TimelineDomains.class);
       }
     } catch (Exception e) {
       LOG.error("Error when reading  " + e.getMessage());


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


[21/50] [abbrv] hadoop git commit: YARN-4765 Split TestHBaseTimelineStorage into multiple test classes (Varun Saxena via Vrushali C)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/022bf783/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
new file mode 100644
index 0000000..e70198a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
@@ -0,0 +1,1849 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests for apps stored in TimelineStorage.
+ */
+public class TestHBaseTimelineStorageApps {
+
+  private static HBaseTestingUtility util;
+  private HBaseTimelineReaderImpl reader;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    util = new HBaseTestingUtility();
+    util.startMiniCluster();
+    createSchema();
+    DataGeneratorForTest.loadApps(util);
+  }
+
+  private static void createSchema() throws IOException {
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
+  }
+
+  @Before
+  public void init() throws Exception {
+    reader = new HBaseTimelineReaderImpl();
+    reader.init(util.getConfiguration());
+    reader.start();
+  }
+
+  @After
+  public void stop() throws Exception {
+    if (reader != null) {
+      reader.stop();
+      reader.close();
+    }
+  }
+
+  private static void matchMetrics(Map<Long, Number> m1, Map<Long, Number> m2) {
+    assertEquals(m1.size(), m2.size());
+    for (Map.Entry<Long, Number> entry : m2.entrySet()) {
+      Number val = m1.get(entry.getKey());
+      assertNotNull(val);
+      assertEquals(val.longValue(), entry.getValue().longValue());
+    }
+  }
+
+  private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
+      String user, String flow, Long runid, String appName) {
+
+    ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
+
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(user, key.getUserId());
+    assertEquals(flow, key.getFlowName());
+    assertEquals(runid, key.getFlowRunId());
+    assertEquals(appName, key.getAppId());
+    return true;
+  }
+
+  @Test
+  public void testWriteNullApplicationToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    ApplicationEntity entity = new ApplicationEntity();
+    String appId = "application_1000178881110_2002";
+    entity.setId(appId);
+    long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("in fo M apK  ey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
+    te.addEntity(entity);
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_check_null_application";
+      String user = "user1check_null_application";
+      //set the flow name to null
+      String flow = null;
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+      hbi.stop();
+
+      // retrieve the row
+      Scan scan = new Scan();
+      scan.setStartRow(Bytes.toBytes(cluster));
+      scan.setStopRow(Bytes.toBytes(cluster + "1"));
+      Connection conn = ConnectionFactory.createConnection(c1);
+      ResultScanner resultScanner = new ApplicationTable()
+          .getResultScanner(c1, conn, scan);
+
+      assertTrue(resultScanner != null);
+      // try to iterate over results
+      int count = 0;
+      for (Result rr = resultScanner.next(); rr != null;
+          rr = resultScanner.next()) {
+        count++;
+      }
+      // there should be no rows written
+      // no exceptions thrown during write
+      assertEquals(0, count);
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Test
+  public void testWriteApplicationToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    ApplicationEntity entity = new ApplicationEntity();
+    String appId = "application_1000178881110_2002";
+    entity.setId(appId);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    long ts = System.currentTimeMillis();
+    metricValues.put(ts - 120000, 100000000);
+    metricValues.put(ts - 100000, 200000000);
+    metricValues.put(ts - 80000, 300000000);
+    metricValues.put(ts - 60000, 400000000);
+    metricValues.put(ts - 40000, 50000000000L);
+    metricValues.put(ts - 20000, 60000000000L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    // add aggregated metrics
+    TimelineEntity aggEntity = new TimelineEntity();
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    aggEntity.setId(appId);
+    aggEntity.setType(type);
+    long cTime2 = 1425016502000L;
+    aggEntity.setCreatedTime(cTime2);
+
+    TimelineMetric aggMetric = new TimelineMetric();
+    aggMetric.setId("MEM_USAGE");
+    Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
+    long aggTs = ts;
+    aggMetricValues.put(aggTs - 120000, 102400000L);
+    aggMetric.setType(Type.SINGLE_VALUE);
+    aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+    aggMetric.setValues(aggMetricValues);
+    Set<TimelineMetric> aggMetrics = new HashSet<>();
+    aggMetrics.add(aggMetric);
+    entity.addMetrics(aggMetrics);
+    te.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_write_app";
+      String user = "user1";
+      String flow = "s!ome_f\tlow  _n am!e";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+
+      // Write entity again, this time without created time.
+      entity = new ApplicationEntity();
+      appId = "application_1000178881110_2002";
+      entity.setId(appId);
+      // add the info map in Timeline Entity
+      Map<String, Object> infoMap1 = new HashMap<>();
+      infoMap1.put("infoMapKey3", "infoMapValue1");
+      entity.addInfo(infoMap1);
+      te = new TimelineEntities();
+      te.addEntity(entity);
+      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+      hbi.stop();
+
+      infoMap.putAll(infoMap1);
+      // retrieve the row
+      ApplicationRowKey applicationRowKey =
+          new ApplicationRowKey(cluster, user, flow, runid, appId);
+      byte[] rowKey = applicationRowKey.getRowKey();
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      Result result = new ApplicationTable().getResult(c1, conn, get);
+
+      assertTrue(result != null);
+      assertEquals(17, result.size());
+
+      // check the row key
+      byte[] row1 = result.getRow();
+      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
+          appId));
+
+      // check info column family
+      String id1 = ApplicationColumn.ID.readResult(result).toString();
+      assertEquals(appId, id1);
+
+      Long cTime1 =
+          (Long) ApplicationColumn.CREATED_TIME.readResult(result);
+      assertEquals(cTime, cTime1);
+
+      Map<String, Object> infoColumns =
+          ApplicationColumnPrefix.INFO.readResults(result,
+              new StringKeyConverter());
+      assertEquals(infoMap, infoColumns);
+
+      // Remember isRelatedTo is of type Map<String, Set<String>>
+      for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
+          .entrySet()) {
+        Object isRelatedToValue =
+            ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
+                isRelatedToEntry.getKey());
+        String compoundValue = isRelatedToValue.toString();
+        // id7?id9?id6
+        Set<String> isRelatedToValues =
+            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+        assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
+            isRelatedToValues.size());
+        for (String v : isRelatedToEntry.getValue()) {
+          assertTrue(isRelatedToValues.contains(v));
+        }
+      }
+
+      // RelatesTo
+      for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
+          .entrySet()) {
+        String compoundValue =
+            ApplicationColumnPrefix.RELATES_TO.readResult(result,
+                relatesToEntry.getKey()).toString();
+        // id3?id4?id5
+        Set<String> relatesToValues =
+            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+        assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
+            relatesToValues.size());
+        for (String v : relatesToEntry.getValue()) {
+          assertTrue(relatesToValues.contains(v));
+        }
+      }
+
+      KeyConverter<String> stringKeyConverter = new StringKeyConverter();
+      // Configuration
+      Map<String, Object> configColumns =
+          ApplicationColumnPrefix.CONFIG
+              .readResults(result, stringKeyConverter);
+      assertEquals(conf, configColumns);
+
+      NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result,
+              stringKeyConverter);
+
+      NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
+      matchMetrics(metricValues, metricMap);
+
+      // read the timeline entity using the reader this time. In metrics limit
+      // specify Integer MAX_VALUE. A TIME_SERIES will be returned(if more than
+      // one value exists for a metric).
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appId,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null,
+          EnumSet.of(TimelineReader.Field.ALL), Integer.MAX_VALUE));
+      assertNotNull(e1);
+
+      // verify attributes
+      assertEquals(appId, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      Map<String, Object> infoMap2 = e1.getInfo();
+      assertEquals(infoMap, infoMap2);
+
+      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
+      assertEquals(isRelatedTo, isRelatedTo2);
+
+      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
+      assertEquals(relatesTo, relatesTo2);
+
+      Map<String, String> conf2 = e1.getConfigs();
+      assertEquals(conf, conf2);
+
+      Set<TimelineMetric> metrics2 = e1.getMetrics();
+      assertEquals(2, metrics2.size());
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
+            metric2.getId().equals("MEM_USAGE"));
+        if (metric2.getId().equals("MAP_SLOT_MILLIS")) {
+          assertEquals(6, metricValues2.size());
+          matchMetrics(metricValues, metricValues2);
+        }
+        if (metric2.getId().equals("MEM_USAGE")) {
+          assertEquals(1, metricValues2.size());
+          matchMetrics(aggMetricValues, metricValues2);
+        }
+      }
+
+      // In metrics limit specify a value of 3. No more than 3 values for a
+      // metric will be returned.
+      e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
+          runid, appId, entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null,
+          EnumSet.of(TimelineReader.Field.ALL), 3));
+      assertNotNull(e1);
+      assertEquals(appId, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(conf, e1.getConfigs());
+      metrics2 = e1.getMetrics();
+      assertEquals(2, metrics2.size());
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertTrue(metricValues2.size() <= 3);
+        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
+            metric2.getId().equals("MEM_USAGE"));
+      }
+
+      // Check if single value(latest value) instead of time series is returned
+      // if metricslimit is not set(null), irrespective of number of metric
+      // values.
+      e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appId,
+         entity.getType(), entity.getId()), new TimelineDataToRetrieve(
+         null, null, EnumSet.of(TimelineReader.Field.ALL), null));
+      assertNotNull(e1);
+      assertEquals(appId, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(infoMap, e1.getInfo());
+      assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
+      assertEquals(relatesTo, e1.getRelatesToEntities());
+      assertEquals(conf, e1.getConfigs());
+      assertEquals(2, e1.getMetrics().size());
+      for (TimelineMetric metric : e1.getMetrics()) {
+        assertEquals(1, metric.getValues().size());
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertTrue(metric.getId().equals("MAP_SLOT_MILLIS") ||
+            metric.getId().equals("MEM_USAGE"));
+        assertEquals(1, metric.getValues().size());
+        if (metric.getId().equals("MAP_SLOT_MILLIS")) {
+          assertTrue(metric.getValues().containsKey(ts - 20000));
+          assertEquals(metricValues.get(ts - 20000),
+              metric.getValues().get(ts - 20000));
+        }
+        if (metric.getId().equals("MEM_USAGE")) {
+          assertTrue(metric.getValues().containsKey(aggTs - 120000));
+          assertEquals(aggMetricValues.get(aggTs - 120000),
+              metric.getValues().get(aggTs - 120000));
+        }
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Test
+  public void testEvents() throws IOException {
+    TimelineEvent event = new TimelineEvent();
+    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
+    event.setId(eventId);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+
+    final TimelineEntity entity = new ApplicationEntity();
+    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.addEvent(event);
+
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_events";
+      String user = "user2";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = "application_123465899910_1001";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.stop();
+
+      // retrieve the row
+      ApplicationRowKey applicationRowKey =
+          new ApplicationRowKey(cluster, user, flow, runid, appName);
+      byte[] rowKey = applicationRowKey.getRowKey();
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      Result result = new ApplicationTable().getResult(c1, conn, get);
+
+      assertTrue(result != null);
+
+      // check the row key
+      byte[] row1 = result.getRow();
+      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
+          appName));
+
+      Map<EventColumnName, Object> eventsResult =
+          ApplicationColumnPrefix.EVENT.readResults(result,
+              new EventColumnNameConverter());
+      // there should be only one event
+      assertEquals(1, eventsResult.size());
+      for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
+        EventColumnName eventColumnName = e.getKey();
+        // the qualifier is a compound key
+        // hence match individual values
+        assertEquals(eventId, eventColumnName.getId());
+        assertEquals(expTs, eventColumnName.getTimestamp());
+        assertEquals(expKey, eventColumnName.getInfoKey());
+        Object value = e.getValue();
+        // there should be only one timestamp and value
+        assertEquals(expVal, value.toString());
+      }
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      TimelineEntity e2 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, null, null, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      assertNotNull(e1);
+      assertNotNull(e2);
+      assertEquals(e1, e2);
+
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, Long.valueOf(e.getTimestamp()));
+        Map<String, Object> info = e.getInfo();
+        assertEquals(1, info.size());
+        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
+          assertEquals(expKey, infoEntry.getKey());
+          assertEquals(expVal, infoEntry.getValue());
+        }
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Test
+  public void testNonIntegralMetricValues() throws IOException {
+    TimelineEntities teApp = new TimelineEntities();
+    ApplicationEntity entityApp = new ApplicationEntity();
+    String appId = "application_1000178881110_2002";
+    entityApp.setId(appId);
+    entityApp.setCreatedTime(1425016501000L);
+    // add metrics with floating point values
+    Set<TimelineMetric> metricsApp = new HashSet<>();
+    TimelineMetric mApp = new TimelineMetric();
+    mApp.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricAppValues = new HashMap<Long, Number>();
+    long ts = System.currentTimeMillis();
+    metricAppValues.put(ts - 20, 10.5);
+    metricAppValues.put(ts - 10, 20.5);
+    mApp.setType(Type.TIME_SERIES);
+    mApp.setValues(metricAppValues);
+    metricsApp.add(mApp);
+    entityApp.addMetrics(metricsApp);
+    teApp.addEntity(entityApp);
+
+    TimelineEntities teEntity = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId("hello");
+    entity.setType("world");
+    entity.setCreatedTime(1425016501000L);
+    // add metrics with floating point values
+    Set<TimelineMetric> metricsEntity = new HashSet<>();
+    TimelineMetric mEntity = new TimelineMetric();
+    mEntity.setId("MAP_SLOT_MILLIS");
+    mEntity.addValue(ts - 20, 10.5);
+    metricsEntity.add(mEntity);
+    entity.addMetrics(metricsEntity);
+    teEntity.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      // Writing application entity.
+      try {
+        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teApp);
+        Assert.fail("Expected an exception as metric values are non integral");
+      } catch (IOException e) {}
+
+      // Writing generic entity.
+      try {
+        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teEntity);
+        Assert.fail("Expected an exception as metric values are non integral");
+      } catch (IOException e) {}
+      hbi.stop();
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Test
+  public void testReadApps() throws Exception {
+    TimelineEntity entity = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertNotNull(entity);
+    assertEquals(3, entity.getConfigs().size());
+    assertEquals(1, entity.getIsRelatedToEntities().size());
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(3, entities.size());
+    int cfgCnt = 0;
+    int metricCnt = 0;
+    int infoCnt = 0;
+    int eventCnt = 0;
+    int relatesToCnt = 0;
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
+          timelineEntity.getConfigs().size();
+      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
+          timelineEntity.getMetrics().size();
+      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
+          timelineEntity.getInfo().size();
+      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
+          timelineEntity.getEvents().size();
+      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
+          timelineEntity.getRelatesToEntities().size();
+      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
+          timelineEntity.getIsRelatedToEntities().size();
+    }
+    assertEquals(5, cfgCnt);
+    assertEquals(3, metricCnt);
+    assertEquals(5, infoCnt);
+    assertEquals(4, eventCnt);
+    assertEquals(4, relatesToCnt);
+    assertEquals(4, isRelatedToCnt);
+  }
+
+  @Test
+  public void testFilterAppsByCreatedTime() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
+        null, null, null, null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(3, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("application_1111111111_2222") &&
+          !entity.getId().equals("application_1111111111_3333") &&
+          !entity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entities with ids' application_1111111111_2222, " +
+            "application_1111111111_3333 and application_1111111111_4444" +
+            " should be present");
+      }
+    }
+    entities =  reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("application_1111111111_3333") &&
+          !entity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Apps with ids' application_1111111111_3333 and" +
+            " application_1111111111_4444 should be present");
+      }
+    }
+    entities =  reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("App with id application_1111111111_2222 should" +
+            " be present");
+      }
+    }
+  }
+
+  @Test
+  public void testReadAppsDefaultView() throws Exception {
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineDataToRetrieve());
+    assertNotNull(e1);
+    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
+        e1.getRelatesToEntities().isEmpty());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve());
+    assertEquals(3, es1.size());
+    for (TimelineEntity e : es1) {
+      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
+          e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
+          e.getRelatesToEntities().isEmpty());
+    }
+  }
+
+  @Test
+  public void testReadAppsByFields() throws Exception {
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineDataToRetrieve(
+        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
+    assertNotNull(e1);
+    assertEquals(3, e1.getConfigs().size());
+    assertEquals(0, e1.getIsRelatedToEntities().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(
+        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
+    assertEquals(3, es1.size());
+    int metricsCnt = 0;
+    int isRelatedToCnt = 0;
+    int infoCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricsCnt += entity.getMetrics().size();
+      isRelatedToCnt += entity.getIsRelatedToEntities().size();
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(0, infoCnt);
+    assertEquals(4, isRelatedToCnt);
+    assertEquals(3, metricsCnt);
+  }
+
+  @Test
+  public void testReadAppsIsRelatedTo() throws Exception {
+    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_3333");
+      }
+    }
+    assertEquals(3, isRelatedToCnt);
+
+    TimelineFilterList irt1 = new TimelineFilterList();
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity id should have been application_1111111111_4444");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_3333");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt3 = new TimelineFilterList();
+    irt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt4 = new TimelineFilterList();
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList irt5 = new TimelineFilterList();
+    irt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+  }
+
+
+  @Test
+  public void testReadAppsRelatesTo() throws Exception {
+    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_4444");
+      }
+    }
+    assertEquals(3, relatesToCnt);
+
+    TimelineFilterList rt1 = new TimelineFilterList();
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_4444");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt3 = new TimelineFilterList();
+    rt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt4 = new TimelineFilterList();
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList rt5 = new TimelineFilterList();
+    rt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList list3 = new TimelineFilterList();
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list4 = new TimelineFilterList();
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto2"))));
+    TimelineFilterList combinedList =
+        new TimelineFilterList(Operator.OR, list3, list4);
+    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
+        new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadAppsRelationsAndEventFiltersDefaultView()
+      throws Exception {
+    TimelineFilterList eventFilter = new TimelineFilterList();
+    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
+        "end_event"));
+    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList isRelatedTo = new TimelineFilterList();
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
+        null, null, null, eventFilter),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    int isRelatedToCnt = 0;
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity id should have been application_1111111111_4444");
+      }
+    }
+    assertEquals(0, eventCnt);
+    assertEquals(0, isRelatedToCnt);
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadAppsConfigFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(2, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    TimelineFilterList confFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(1, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(3, cfgCnt);
+
+    TimelineFilterList confFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_param2", "value2"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList2, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList3, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList4, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList5, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadAppsEventFilters() throws Exception {
+    TimelineFilterList ef = new TimelineFilterList();
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity id should have been application_1111111111_4444");
+      }
+    }
+    assertEquals(1, eventCnt);
+
+    TimelineFilterList ef1 = new TimelineFilterList();
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef1), new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity id should have been application_1111111111_4444");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef2 = new TimelineFilterList();
+    ef2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef2),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_4444");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef3 = new TimelineFilterList();
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef3),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "start_event"));
+    TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef4),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef5 = new TimelineFilterList();
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "update_event"));
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef5),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
+    }
+    assertEquals(0, eventCnt);
+  }
+
+  @Test
+  public void testReadAppsConfigPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineDataToRetrieve(list, null, null, null));
+    assertNotNull(e1);
+    assertEquals(1, e1.getConfigs().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null) ,
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(list, null, null, null));
+    int cfgCnt = 0;
+    for (TimelineEntity entity : es1) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(3, cfgCnt);
+  }
+
+  @Test
+  public void testReadAppsConfigFilterPrefix() throws Exception {
+    TimelineFilterList confFilterList = new TimelineFilterList();
+    confFilterList.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(list, null, null, null));
+    assertEquals(1, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(2, cfgCnt);
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confsToRetrieve =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_"));
+    TimelineFilterList confFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with config_",
+            confKey.startsWith("config_"));
+      }
+    }
+    assertEquals(2, cfgCnt);
+  }
+
+  @Test
+  public void testReadAppsMetricFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(2, entities.size());
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    TimelineFilterList metricFilterList1 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L),
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(1, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(2, metricCnt);
+
+    TimelineFilterList metricFilterList2 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 40000000000L),
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList2, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList3 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "dummy_metric", 5));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList3, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList4 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList4, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList5 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5, false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList5, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadAppsMetricPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineDataToRetrieve(null, list, null, null));
+    assertNotNull(e1);
+    assertEquals(1, e1.getMetrics().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(null, list, null, null));
+    int metricCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+  }
+
+  @Test
+  public void testReadAppsMetricFilterPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineFilterList metricFilterList = new TimelineFilterList();
+    metricFilterList.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 0L));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, list, null, null));
+    int metricCnt = 0;
+    assertEquals(1, entities.size());
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(1, metricCnt);
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineFilterList metricFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
+    metricCnt = 0;
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+
+    entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null), new TimelineDataToRetrieve(null,
+        metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
+    metricCnt = 0;
+    int metricValCnt = 0;
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        metricValCnt += metric.getValues().size();
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+    assertEquals(7, metricValCnt);
+  }
+
+  @Test
+  public void testReadAppsInfoFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey3", 85.85));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue1"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey2", 10));
+    TimelineFilterList infoFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(2, entities.size());
+    int infoCnt = 0;
+    for (TimelineEntity entity : entities) {
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(5, infoCnt);
+
+    TimelineFilterList infoFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(1, entities.size());
+    infoCnt = 0;
+    for (TimelineEntity entity : entities) {
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(3, infoCnt);
+
+    TimelineFilterList infoFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue2"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey3", 85.85));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_info", "some_value"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+    assertEquals(3, entities.size());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}


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


[43/50] [abbrv] hadoop git commit: HADOOP-13763. KMS REST API Documentation Decrypt URL typo. Contributed by Jeffrey E Rodriguez.

Posted by vi...@apache.org.
HADOOP-13763. KMS REST API Documentation Decrypt URL typo. Contributed by Jeffrey E Rodriguez.


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

Branch: refs/heads/HDFS-9806
Commit: b62bc2bbd80bb751348f0c1f655d5e456624663e
Parents: 4df8ed6
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Oct 27 18:05:40 2016 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Oct 27 18:05:40 2016 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b62bc2bb/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
index 0c6d0b2..69eb1dd 100644
--- a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
+++ b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
@@ -896,7 +896,7 @@ $H4 Decrypt Encrypted Key
 
 *REQUEST:*
 
-    POST http://HOST:PORT/kms/v1/keyversion/<version-name>/_eek?ee_op=decrypt
+    POST http://HOST:PORT/kms/v1/keyversion/<version-name>/_eek?eek_op=decrypt
     Content-Type: application/json
 
     {


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


[05/50] [abbrv] hadoop git commit: HADOOP-13201. Print the directory paths when ViewFs denies the rename operation on internal dirs. Contributed by Tianyin Xiu

Posted by vi...@apache.org.
HADOOP-13201. Print the directory paths when ViewFs denies the rename operation on internal dirs. Contributed by Tianyin Xiu


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

Branch: refs/heads/HDFS-9806
Commit: 0c837db8a874079dd5db83a7eef9c4d2b9d0e9ff
Parents: b439517
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Oct 27 19:46:41 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Thu Oct 27 19:46:41 2016 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java    | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c837db8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
index 3beda53..6195540 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
@@ -472,14 +472,16 @@ public class ViewFs extends AbstractFileSystem {
   
     if (resSrc.isInternalDir()) {
       throw new AccessControlException(
-          "Cannot Rename within internal dirs of mount table: it is readOnly");
+          "Cannot Rename within internal dirs of mount table: src=" + src
+              + " is readOnly");
     }
-      
+
     InodeTree.ResolveResult<AbstractFileSystem> resDst = 
                                 fsState.resolve(getUriPath(dst), false);
     if (resDst.isInternalDir()) {
       throw new AccessControlException(
-          "Cannot Rename within internal dirs of mount table: it is readOnly");
+          "Cannot Rename within internal dirs of mount table: dest=" + dst
+              + " is readOnly");
     }
     
     /**


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


[30/50] [abbrv] hadoop git commit: HADOOP-10075. Update jetty dependency to version 9 (rkanter)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877f20f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js
new file mode 100644
index 0000000..d4d8985
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js
@@ -0,0 +1,4544 @@
+/*
+ * jsTree 1.0-rc3
+ * http://jstree.com/
+ *
+ * Copyright (c) 2010 Ivan Bozhanov (vakata.com)
+ *
+ * Licensed same as jquery - under the terms of either the MIT License or the GPL Version 2 License
+ *   http://www.opensource.org/licenses/mit-license.php
+ *   http://www.gnu.org/licenses/gpl.html
+ *
+ * $Date$
+ * $Revision$
+ */
+
+/*jslint browser: true, onevar: true, undef: true, bitwise: true, strict: true */
+/*global window : false, clearInterval: false, clearTimeout: false, document: false, setInterval: false, setTimeout: false, jQuery: false, navigator: false, XSLTProcessor: false, DOMParser: false, XMLSerializer: false*/
+
+"use strict";
+
+// top wrapper to prevent multiple inclusion (is this OK?)
+(function () { if(jQuery && jQuery.jstree) { return; }
+  var is_ie6 = false, is_ie7 = false, is_ff2 = false;
+
+/*
+ * jsTree core
+ */
+(function ($) {
+  // Common functions not related to jsTree
+  // decided to move them to a `vakata` "namespace"
+  $.vakata = {};
+  // CSS related functions
+  $.vakata.css = {
+    get_css : function(rule_name, delete_flag, sheet) {
+      rule_name = rule_name.toLowerCase();
+      var css_rules = sheet.cssRules || sheet.rules,
+        j = 0;
+      do {
+        if(css_rules.length && j > css_rules.length + 5) { return false; }
+        if(css_rules[j].selectorText && css_rules[j].selectorText.toLowerCase() == rule_name) {
+          if(delete_flag === true) {
+            if(sheet.removeRule) { sheet.removeRule(j); }
+            if(sheet.deleteRule) { sheet.deleteRule(j); }
+            return true;
+          }
+          else { return css_rules[j]; }
+        }
+      }
+      while (css_rules[++j]);
+      return false;
+    },
+    add_css : function(rule_name, sheet) {
+      if($.jstree.css.get_css(rule_name, false, sheet)) { return false; }
+      if(sheet.insertRule) { sheet.insertRule(rule_name + ' { }', 0); } else { sheet.addRule(rule_name, null, 0); }
+      return $.vakata.css.get_css(rule_name);
+    },
+    remove_css : function(rule_name, sheet) {
+      return $.vakata.css.get_css(rule_name, true, sheet);
+    },
+    add_sheet : function(opts) {
+      var tmp = false, is_new = true;
+      if(opts.str) {
+        if(opts.title) { tmp = $("style[id='" + opts.title + "-stylesheet']")[0]; }
+        if(tmp) { is_new = false; }
+        else {
+          tmp = document.createElement("style");
+          tmp.setAttribute('type',"text/css");
+          if(opts.title) { tmp.setAttribute("id", opts.title + "-stylesheet"); }
+        }
+        if(tmp.styleSheet) {
+          if(is_new) {
+            document.getElementsByTagName("head")[0].appendChild(tmp);
+            tmp.styleSheet.cssText = opts.str;
+          }
+          else {
+            tmp.styleSheet.cssText = tmp.styleSheet.cssText + " " + opts.str;
+          }
+        }
+        else {
+          tmp.appendChild(document.createTextNode(opts.str));
+          document.getElementsByTagName("head")[0].appendChild(tmp);
+        }
+        return tmp.sheet || tmp.styleSheet;
+      }
+      if(opts.url) {
+        if(document.createStyleSheet) {
+          try { tmp = document.createStyleSheet(opts.url); } catch (e) { }
+        }
+        else {
+          tmp      = document.createElement('link');
+          tmp.rel    = 'stylesheet';
+          tmp.type  = 'text/css';
+          tmp.media  = "all";
+          tmp.href  = opts.url;
+          document.getElementsByTagName("head")[0].appendChild(tmp);
+          return tmp.styleSheet;
+        }
+      }
+    }
+  };
+
+  // private variables
+  var instances = [],      // instance array (used by $.jstree.reference/create/focused)
+    focused_instance = -1,  // the index in the instance array of the currently focused instance
+    plugins = {},      // list of included plugins
+    prepared_move = {};    // for the move_node function
+
+  // jQuery plugin wrapper (thanks to jquery UI widget function)
+  $.fn.jstree = function (settings) {
+    var isMethodCall = (typeof settings == 'string'), // is this a method call like $().jstree("open_node")
+      args = Array.prototype.slice.call(arguments, 1),
+      returnValue = this;
+
+    // if a method call execute the method on all selected instances
+    if(isMethodCall) {
+      if(settings.substring(0, 1) == '_') { return returnValue; }
+      this.each(function() {
+        var instance = instances[$.data(this, "jstree-instance-id")],
+          methodValue = (instance && $.isFunction(instance[settings])) ? instance[settings].apply(instance, args) : instance;
+          if(typeof methodValue !== "undefined" && (settings.indexOf("is_") === 0 || (methodValue !== true && methodValue !== false))) { returnValue = methodValue; return false; }
+      });
+    }
+    else {
+      this.each(function() {
+        // extend settings and allow for multiple hashes and $.data
+        var instance_id = $.data(this, "jstree-instance-id"),
+          a = [],
+          b = settings ? $.extend({}, true, settings) : {},
+          c = $(this),
+          s = false,
+          t = [];
+        a = a.concat(args);
+        if(c.data("jstree")) { a.push(c.data("jstree")); }
+        b = a.length ? $.extend.apply(null, [true, b].concat(a)) : b;
+
+        // if an instance already exists, destroy it first
+        if(typeof instance_id !== "undefined" && instances[instance_id]) { instances[instance_id].destroy(); }
+        // push a new empty object to the instances array
+        instance_id = parseInt(instances.push({}),10) - 1;
+        // store the jstree instance id to the container element
+        $.data(this, "jstree-instance-id", instance_id);
+        // clean up all plugins
+        b.plugins = $.isArray(b.plugins) ? b.plugins : $.jstree.defaults.plugins.slice();
+        b.plugins.unshift("core");
+        // only unique plugins
+        b.plugins = b.plugins.sort().join(",,").replace(/(,|^)([^,]+)(,,\2)+(,|$)/g,"$1$2$4").replace(/,,+/g,",").replace(/,$/,"").split(",");
+
+        // extend defaults with passed data
+        s = $.extend(true, {}, $.jstree.defaults, b);
+        s.plugins = b.plugins;
+        $.each(plugins, function (i, val) {
+          if($.inArray(i, s.plugins) === -1) { s[i] = null; delete s[i]; }
+          else { t.push(i); }
+        });
+        s.plugins = t;
+
+        // push the new object to the instances array (at the same time set the default classes to the container) and init
+        instances[instance_id] = new $.jstree._instance(instance_id, $(this).addClass("jstree jstree-" + instance_id), s);
+        // init all activated plugins for this instance
+        $.each(instances[instance_id]._get_settings().plugins, function (i, val) { instances[instance_id].data[val] = {}; });
+        $.each(instances[instance_id]._get_settings().plugins, function (i, val) { if(plugins[val]) { plugins[val].__init.apply(instances[instance_id]); } });
+        // initialize the instance
+        setTimeout(function() { instances[instance_id].init(); }, 0);
+      });
+    }
+    // return the jquery selection (or if it was a method call that returned a value - the returned value)
+    return returnValue;
+  };
+  // object to store exposed functions and objects
+  $.jstree = {
+    defaults : {
+      plugins : []
+    },
+    _focused : function () { return instances[focused_instance] || null; },
+    _reference : function (needle) {
+      // get by instance id
+      if(instances[needle]) { return instances[needle]; }
+      // get by DOM (if still no luck - return null
+      var o = $(needle);
+      if(!o.length && typeof needle === "string") { o = $("#" + needle); }
+      if(!o.length) { return null; }
+      return instances[o.closest(".jstree").data("jstree-instance-id")] || null;
+    },
+    _instance : function (index, container, settings) {
+      // for plugins to store data in
+      this.data = { core : {} };
+      this.get_settings  = function () { return $.extend(true, {}, settings); };
+      this._get_settings  = function () { return settings; };
+      this.get_index    = function () { return index; };
+      this.get_container  = function () { return container; };
+      this.get_container_ul = function () { return container.children("ul:eq(0)"); };
+      this._set_settings  = function (s) {
+        settings = $.extend(true, {}, settings, s);
+      };
+    },
+    _fn : { },
+    plugin : function (pname, pdata) {
+      pdata = $.extend({}, {
+        __init    : $.noop,
+        __destroy  : $.noop,
+        _fn      : {},
+        defaults  : false
+      }, pdata);
+      plugins[pname] = pdata;
+
+      $.jstree.defaults[pname] = pdata.defaults;
+      $.each(pdata._fn, function (i, val) {
+        val.plugin    = pname;
+        val.old      = $.jstree._fn[i];
+        $.jstree._fn[i] = function () {
+          var rslt,
+            func = val,
+            args = Array.prototype.slice.call(arguments),
+            evnt = new $.Event("before.jstree"),
+            rlbk = false;
+
+          if(this.data.core.locked === true && i !== "unlock" && i !== "is_locked") { return; }
+
+          // Check if function belongs to the included plugins of this instance
+          do {
+            if(func && func.plugin && $.inArray(func.plugin, this._get_settings().plugins) !== -1) { break; }
+            func = func.old;
+          } while(func);
+          if(!func) { return; }
+
+          // context and function to trigger events, then finally call the function
+          if(i.indexOf("_") === 0) {
+            rslt = func.apply(this, args);
+          }
+          else {
+            rslt = this.get_container().triggerHandler(evnt, { "func" : i, "inst" : this, "args" : args, "plugin" : func.plugin });
+            if(rslt === false) { return; }
+            if(typeof rslt !== "undefined") { args = rslt; }
+
+            rslt = func.apply(
+              $.extend({}, this, {
+                __callback : function (data) {
+                  this.get_container().triggerHandler( i + '.jstree', { "inst" : this, "args" : args, "rslt" : data, "rlbk" : rlbk });
+                },
+                __rollback : function () {
+                  rlbk = this.get_rollback();
+                  return rlbk;
+                },
+                __call_old : function (replace_arguments) {
+                  return func.old.apply(this, (replace_arguments ? Array.prototype.slice.call(arguments, 1) : args ) );
+                }
+              }), args);
+          }
+
+          // return the result
+          return rslt;
+        };
+        $.jstree._fn[i].old = val.old;
+        $.jstree._fn[i].plugin = pname;
+      });
+    },
+    rollback : function (rb) {
+      if(rb) {
+        if(!$.isArray(rb)) { rb = [ rb ]; }
+        $.each(rb, function (i, val) {
+          instances[val.i].set_rollback(val.h, val.d);
+        });
+      }
+    }
+  };
+  // set the prototype for all instances
+  $.jstree._fn = $.jstree._instance.prototype = {};
+
+  // load the css when DOM is ready
+  $(function() {
+    // code is copied from jQuery ($.browser is deprecated + there is a bug in IE)
+    var u = navigator.userAgent.toLowerCase(),
+      v = (u.match( /.+?(?:rv|it|ra|ie)[\/: ]([\d.]+)/ ) || [0,'0'])[1],
+      css_string = '' +
+        '.jstree ul, .jstree li { display:block; margin:0 0 0 0; padding:0 0 0 0; list-style-type:none; } ' +
+        '.jstree li { display:block; min-height:18px; line-height:18px; white-space:nowrap; margin-left:18px; min-width:18px; } ' +
+        '.jstree-rtl li { margin-left:0; margin-right:18px; } ' +
+        '.jstree > ul > li { margin-left:0px; } ' +
+        '.jstree-rtl > ul > li { margin-right:0px; } ' +
+        '.jstree ins { display:inline-block; text-decoration:none; width:18px; height:18px; margin:0 0 0 0; padding:0; } ' +
+        '.jstree a { display:inline-block; line-height:16px; height:16px; color:black; white-space:nowrap; text-decoration:none; padding:1px 2px; margin:0; } ' +
+        '.jstree a:focus { outline: none; } ' +
+        '.jstree a > ins { height:16px; width:16px; } ' +
+        '.jstree a > .jstree-icon { margin-right:3px; } ' +
+        '.jstree-rtl a > .jstree-icon { margin-left:3px; margin-right:0; } ' +
+        'li.jstree-open > ul { display:block; } ' +
+        'li.jstree-closed > ul { display:none; } ';
+    // Correct IE 6 (does not support the > CSS selector)
+    if(/msie/.test(u) && parseInt(v, 10) == 6) {
+      is_ie6 = true;
+
+      // fix image flicker and lack of caching
+      try {
+        document.execCommand("BackgroundImageCache", false, true);
+      } catch (err) { }
+
+      css_string += '' +
+        '.jstree li { height:18px; margin-left:0; margin-right:0; } ' +
+        '.jstree li li { margin-left:18px; } ' +
+        '.jstree-rtl li li { margin-left:0px; margin-right:18px; } ' +
+        'li.jstree-open ul { display:block; } ' +
+        'li.jstree-closed ul { display:none !important; } ' +
+        '.jstree li a { display:inline; border-width:0 !important; padding:0px 2px !important; } ' +
+        '.jstree li a ins { height:16px; width:16px; margin-right:3px; } ' +
+        '.jstree-rtl li a ins { margin-right:0px; margin-left:3px; } ';
+    }
+    // Correct IE 7 (shifts anchor nodes onhover)
+    if(/msie/.test(u) && parseInt(v, 10) == 7) {
+      is_ie7 = true;
+      css_string += '.jstree li a { border-width:0 !important; padding:0px 2px !important; } ';
+    }
+    // correct ff2 lack of display:inline-block
+    if(!/compatible/.test(u) && /mozilla/.test(u) && parseFloat(v, 10) < 1.9) {
+      is_ff2 = true;
+      css_string += '' +
+        '.jstree ins { display:-moz-inline-box; } ' +
+        '.jstree li { line-height:12px; } ' + // WHY??
+        '.jstree a { display:-moz-inline-box; } ' +
+        '.jstree .jstree-no-icons .jstree-checkbox { display:-moz-inline-stack !important; } ';
+        /* this shouldn't be here as it is theme specific */
+    }
+    // the default stylesheet
+    $.vakata.css.add_sheet({ str : css_string, title : "jstree" });
+  });
+
+  // core functions (open, close, create, update, delete)
+  $.jstree.plugin("core", {
+    __init : function () {
+      this.data.core.locked = false;
+      this.data.core.to_open = this.get_settings().core.initially_open;
+      this.data.core.to_load = this.get_settings().core.initially_load;
+    },
+    defaults : {
+      html_titles  : false,
+      animation  : 500,
+      initially_open : [],
+      initially_load : [],
+      open_parents : true,
+      notify_plugins : true,
+      rtl      : false,
+      load_open  : false,
+      strings    : {
+        loading    : "Loading ...",
+        new_node  : "New node",
+        multiple_selection : "Multiple selection"
+      }
+    },
+    _fn : {
+      init  : function () {
+        this.set_focus();
+        if(this._get_settings().core.rtl) {
+          this.get_container().addClass("jstree-rtl").css("direction", "rtl");
+        }
+        this.get_container().html("<ul><li class='jstree-last jstree-leaf'><ins>&#160;</ins><a class='jstree-loading' href='#'><ins class='jstree-icon'>&#160;</ins>" + this._get_string("loading") + "</a></li></ul>");
+        this.data.core.li_height = this.get_container_ul().find("li.jstree-closed, li.jstree-leaf").eq(0).height() || 18;
+
+        this.get_container()
+          .delegate("li > ins", "click.jstree", $.proxy(function (event) {
+              var trgt = $(event.target);
+              if(trgt.is("ins") && event.pageY - trgt.offset().top < this.data.core.li_height) { this.toggle_node(trgt); }
+            }, this))
+          .bind("mousedown.jstree", $.proxy(function () {
+              this.set_focus(); // This used to be setTimeout(set_focus,0) - why?
+            }, this))
+          .bind("dblclick.jstree", function (event) {
+            var sel;
+            if(document.selection && document.selection.empty) { document.selection.empty(); }
+            else {
+              if(window.getSelection) {
+                sel = window.getSelection();
+                try {
+                  sel.removeAllRanges();
+                  sel.collapse();
+                } catch (err) { }
+              }
+            }
+          });
+        if(this._get_settings().core.notify_plugins) {
+          this.get_container()
+            .bind("load_node.jstree", $.proxy(function (e, data) {
+                var o = this._get_node(data.rslt.obj),
+                  t = this;
+                if(o === -1) { o = this.get_container_ul(); }
+                if(!o.length) { return; }
+                o.find("li").each(function () {
+                  var th = $(this);
+                  if(th.data("jstree")) {
+                    $.each(th.data("jstree"), function (plugin, values) {
+                      if(t.data[plugin] && $.isFunction(t["_" + plugin + "_notify"])) {
+                        t["_" + plugin + "_notify"].call(t, th, values);
+                      }
+                    });
+                  }
+                });
+              }, this));
+        }
+        if(this._get_settings().core.load_open) {
+          this.get_container()
+            .bind("load_node.jstree", $.proxy(function (e, data) {
+                var o = this._get_node(data.rslt.obj),
+                  t = this;
+                if(o === -1) { o = this.get_container_ul(); }
+                if(!o.length) { return; }
+                o.find("li.jstree-open:not(:has(ul))").each(function () {
+                  t.load_node(this, $.noop, $.noop);
+                });
+              }, this));
+        }
+        this.__callback();
+        this.load_node(-1, function () { this.loaded(); this.reload_nodes(); });
+      },
+      destroy  : function () {
+        var i,
+          n = this.get_index(),
+          s = this._get_settings(),
+          _this = this;
+
+        $.each(s.plugins, function (i, val) {
+          try { plugins[val].__destroy.apply(_this); } catch(err) { }
+        });
+        this.__callback();
+        // set focus to another instance if this one is focused
+        if(this.is_focused()) {
+          for(i in instances) {
+            if(instances.hasOwnProperty(i) && i != n) {
+              instances[i].set_focus();
+              break;
+            }
+          }
+        }
+        // if no other instance found
+        if(n === focused_instance) { focused_instance = -1; }
+        // remove all traces of jstree in the DOM (only the ones set using jstree*) and cleans all events
+        this.get_container()
+          .unbind(".jstree")
+          .undelegate(".jstree")
+          .removeData("jstree-instance-id")
+          .find("[class^='jstree']")
+            .andSelf()
+            .attr("class", function () { return this.className.replace(/jstree[^ ]*|$/ig,''); });
+        $(document)
+          .unbind(".jstree-" + n)
+          .undelegate(".jstree-" + n);
+        // remove the actual data
+        instances[n] = null;
+        delete instances[n];
+      },
+
+      _core_notify : function (n, data) {
+        if(data.opened) {
+          this.open_node(n, false, true);
+        }
+      },
+
+      lock : function () {
+        this.data.core.locked = true;
+        this.get_container().children("ul").addClass("jstree-locked").css("opacity","0.7");
+        this.__callback({});
+      },
+      unlock : function () {
+        this.data.core.locked = false;
+        this.get_container().children("ul").removeClass("jstree-locked").css("opacity","1");
+        this.__callback({});
+      },
+      is_locked : function () { return this.data.core.locked; },
+      save_opened : function () {
+        var _this = this;
+        this.data.core.to_open = [];
+        this.get_container_ul().find("li.jstree-open").each(function () {
+          if(this.id) { _this.data.core.to_open.push("#" + this.id.toString().replace(/^#/,"").replace(/\\\//g,"/").replace(/\//g,"\\\/").replace(/\\\./g,".").replace(/\./g,"\\.").replace(/\:/g,"\\:")); }
+        });
+        this.__callback(_this.data.core.to_open);
+      },
+      save_loaded : function () { },
+      reload_nodes : function (is_callback) {
+        var _this = this,
+          done = true,
+          current = [],
+          remaining = [];
+        if(!is_callback) {
+          this.data.core.reopen = false;
+          this.data.core.refreshing = true;
+          this.data.core.to_open = $.map($.makeArray(this.data.core.to_open), function (n) { return "#" + n.toString().replace(/^#/,"").replace(/\\\//g,"/").replace(/\//g,"\\\/").replace(/\\\./g,".").replace(/\./g,"\\.").replace(/\:/g,"\\:"); });
+          this.data.core.to_load = $.map($.makeArray(this.data.core.to_load), function (n) { return "#" + n.toString().replace(/^#/,"").replace(/\\\//g,"/").replace(/\//g,"\\\/").replace(/\\\./g,".").replace(/\./g,"\\.").replace(/\:/g,"\\:"); });
+          if(this.data.core.to_open.length) {
+            this.data.core.to_load = this.data.core.to_load.concat(this.data.core.to_open);
+          }
+        }
+        if(this.data.core.to_load.length) {
+          $.each(this.data.core.to_load, function (i, val) {
+            if(val == "#") { return true; }
+            if($(val).length) { current.push(val); }
+            else { remaining.push(val); }
+          });
+          if(current.length) {
+            this.data.core.to_load = remaining;
+            $.each(current, function (i, val) {
+              if(!_this._is_loaded(val)) {
+                _this.load_node(val, function () { _this.reload_nodes(true); }, function () { _this.reload_nodes(true); });
+                done = false;
+              }
+            });
+          }
+        }
+        if(this.data.core.to_open.length) {
+          $.each(this.data.core.to_open, function (i, val) {
+            _this.open_node(val, false, true);
+          });
+        }
+        if(done) {
+          // TODO: find a more elegant approach to syncronizing returning requests
+          if(this.data.core.reopen) { clearTimeout(this.data.core.reopen); }
+          this.data.core.reopen = setTimeout(function () { _this.__callback({}, _this); }, 50);
+          this.data.core.refreshing = false;
+          this.reopen();
+        }
+      },
+      reopen : function () {
+        var _this = this;
+        if(this.data.core.to_open.length) {
+          $.each(this.data.core.to_open, function (i, val) {
+            _this.open_node(val, false, true);
+          });
+        }
+        this.__callback({});
+      },
+      refresh : function (obj) {
+        var _this = this;
+        this.save_opened();
+        if(!obj) { obj = -1; }
+        obj = this._get_node(obj);
+        if(!obj) { obj = -1; }
+        if(obj !== -1) { obj.children("UL").remove(); }
+        else { this.get_container_ul().empty(); }
+        this.load_node(obj, function () { _this.__callback({ "obj" : obj}); _this.reload_nodes(); });
+      },
+      // Dummy function to fire after the first load (so that there is a jstree.loaded event)
+      loaded  : function () {
+        this.__callback();
+      },
+      // deal with focus
+      set_focus  : function () {
+        if(this.is_focused()) { return; }
+        var f = $.jstree._focused();
+        if(f) { f.unset_focus(); }
+
+        this.get_container().addClass("jstree-focused");
+        focused_instance = this.get_index();
+        this.__callback();
+      },
+      is_focused  : function () {
+        return focused_instance == this.get_index();
+      },
+      unset_focus  : function () {
+        if(this.is_focused()) {
+          this.get_container().removeClass("jstree-focused");
+          focused_instance = -1;
+        }
+        this.__callback();
+      },
+
+      // traverse
+      _get_node    : function (obj) {
+        var $obj = $(obj, this.get_container());
+        if($obj.is(".jstree") || obj == -1) { return -1; }
+        $obj = $obj.closest("li", this.get_container());
+        return $obj.length ? $obj : false;
+      },
+      _get_next    : function (obj, strict) {
+        obj = this._get_node(obj);
+        if(obj === -1) { return this.get_container().find("> ul > li:first-child"); }
+        if(!obj.length) { return false; }
+        if(strict) { return (obj.nextAll("li").size() > 0) ? obj.nextAll("li:eq(0)") : false; }
+
+        if(obj.hasClass("jstree-open")) { return obj.find("li:eq(0)"); }
+        else if(obj.nextAll("li").size() > 0) { return obj.nextAll("li:eq(0)"); }
+        else { return obj.parentsUntil(".jstree","li").next("li").eq(0); }
+      },
+      _get_prev    : function (obj, strict) {
+        obj = this._get_node(obj);
+        if(obj === -1) { return this.get_container().find("> ul > li:last-child"); }
+        if(!obj.length) { return false; }
+        if(strict) { return (obj.prevAll("li").length > 0) ? obj.prevAll("li:eq(0)") : false; }
+
+        if(obj.prev("li").length) {
+          obj = obj.prev("li").eq(0);
+          while(obj.hasClass("jstree-open")) { obj = obj.children("ul:eq(0)").children("li:last"); }
+          return obj;
+        }
+        else { var o = obj.parentsUntil(".jstree","li:eq(0)"); return o.length ? o : false; }
+      },
+      _get_parent    : function (obj) {
+        obj = this._get_node(obj);
+        if(obj == -1 || !obj.length) { return false; }
+        var o = obj.parentsUntil(".jstree", "li:eq(0)");
+        return o.length ? o : -1;
+      },
+      _get_children  : function (obj) {
+        obj = this._get_node(obj);
+        if(obj === -1) { return this.get_container().children("ul:eq(0)").children("li"); }
+        if(!obj.length) { return false; }
+        return obj.children("ul:eq(0)").children("li");
+      },
+      get_path    : function (obj, id_mode) {
+        var p = [],
+          _this = this;
+        obj = this._get_node(obj);
+        if(obj === -1 || !obj || !obj.length) { return false; }
+        obj.parentsUntil(".jstree", "li").each(function () {
+          p.push( id_mode ? this.id : _this.get_text(this) );
+        });
+        p.reverse();
+        p.push( id_mode ? obj.attr("id") : this.get_text(obj) );
+        return p;
+      },
+
+      // string functions
+      _get_string : function (key) {
+        return this._get_settings().core.strings[key] || key;
+      },
+
+      is_open    : function (obj) { obj = this._get_node(obj); return obj && obj !== -1 && obj.hasClass("jstree-open"); },
+      is_closed  : function (obj) { obj = this._get_node(obj); return obj && obj !== -1 && obj.hasClass("jstree-closed"); },
+      is_leaf    : function (obj) { obj = this._get_node(obj); return obj && obj !== -1 && obj.hasClass("jstree-leaf"); },
+      correct_state  : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj || obj === -1) { return false; }
+        obj.removeClass("jstree-closed jstree-open").addClass("jstree-leaf").children("ul").remove();
+        this.__callback({ "obj" : obj });
+      },
+      // open/close
+      open_node  : function (obj, callback, skip_animation) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        if(!obj.hasClass("jstree-closed")) { if(callback) { callback.call(); } return false; }
+        var s = skip_animation || is_ie6 ? 0 : this._get_settings().core.animation,
+          t = this;
+        if(!this._is_loaded(obj)) {
+          obj.children("a").addClass("jstree-loading");
+          this.load_node(obj, function () { t.open_node(obj, callback, skip_animation); }, callback);
+        }
+        else {
+          if(this._get_settings().core.open_parents) {
+            obj.parentsUntil(".jstree",".jstree-closed").each(function () {
+              t.open_node(this, false, true);
+            });
+          }
+          if(s) { obj.children("ul").css("display","none"); }
+          obj.removeClass("jstree-closed").addClass("jstree-open").children("a").removeClass("jstree-loading");
+          if(s) { obj.children("ul").stop(true, true).slideDown(s, function () { this.style.display = ""; t.after_open(obj); }); }
+          else { t.after_open(obj); }
+          this.__callback({ "obj" : obj });
+          if(callback) { callback.call(); }
+        }
+      },
+      after_open  : function (obj) { this.__callback({ "obj" : obj }); },
+      close_node  : function (obj, skip_animation) {
+        obj = this._get_node(obj);
+        var s = skip_animation || is_ie6 ? 0 : this._get_settings().core.animation,
+          t = this;
+        if(!obj.length || !obj.hasClass("jstree-open")) { return false; }
+        if(s) { obj.children("ul").attr("style","display:block !important"); }
+        obj.removeClass("jstree-open").addClass("jstree-closed");
+        if(s) { obj.children("ul").stop(true, true).slideUp(s, function () { this.style.display = ""; t.after_close(obj); }); }
+        else { t.after_close(obj); }
+        this.__callback({ "obj" : obj });
+      },
+      after_close  : function (obj) { this.__callback({ "obj" : obj }); },
+      toggle_node  : function (obj) {
+        obj = this._get_node(obj);
+        if(obj.hasClass("jstree-closed")) { return this.open_node(obj); }
+        if(obj.hasClass("jstree-open")) { return this.close_node(obj); }
+      },
+      open_all  : function (obj, do_animation, original_obj) {
+        obj = obj ? this._get_node(obj) : -1;
+        if(!obj || obj === -1) { obj = this.get_container_ul(); }
+        if(original_obj) {
+          obj = obj.find("li.jstree-closed");
+        }
+        else {
+          original_obj = obj;
+          if(obj.is(".jstree-closed")) { obj = obj.find("li.jstree-closed").andSelf(); }
+          else { obj = obj.find("li.jstree-closed"); }
+        }
+        var _this = this;
+        obj.each(function () {
+          var __this = this;
+          if(!_this._is_loaded(this)) { _this.open_node(this, function() { _this.open_all(__this, do_animation, original_obj); }, !do_animation); }
+          else { _this.open_node(this, false, !do_animation); }
+        });
+        // so that callback is fired AFTER all nodes are open
+        if(original_obj.find('li.jstree-closed').length === 0) { this.__callback({ "obj" : original_obj }); }
+      },
+      close_all  : function (obj, do_animation) {
+        var _this = this;
+        obj = obj ? this._get_node(obj) : this.get_container();
+        if(!obj || obj === -1) { obj = this.get_container_ul(); }
+        obj.find("li.jstree-open").andSelf().each(function () { _this.close_node(this, !do_animation); });
+        this.__callback({ "obj" : obj });
+      },
+      clean_node  : function (obj) {
+        obj = obj && obj != -1 ? $(obj) : this.get_container_ul();
+        obj = obj.is("li") ? obj.find("li").andSelf() : obj.find("li");
+        obj.removeClass("jstree-last")
+          .filter("li:last-child").addClass("jstree-last").end()
+          .filter(":has(li)")
+            .not(".jstree-open").removeClass("jstree-leaf").addClass("jstree-closed");
+        obj.not(".jstree-open, .jstree-closed").addClass("jstree-leaf").children("ul").remove();
+        this.__callback({ "obj" : obj });
+      },
+      // rollback
+      get_rollback : function () {
+        this.__callback();
+        return { i : this.get_index(), h : this.get_container().children("ul").clone(true), d : this.data };
+      },
+      set_rollback : function (html, data) {
+        this.get_container().empty().append(html);
+        this.data = data;
+        this.__callback();
+      },
+      // Dummy functions to be overwritten by any datastore plugin included
+      load_node  : function (obj, s_call, e_call) { this.__callback({ "obj" : obj }); },
+      _is_loaded  : function (obj) { return true; },
+
+      // Basic operations: create
+      create_node  : function (obj, position, js, callback, is_loaded) {
+        obj = this._get_node(obj);
+        position = typeof position === "undefined" ? "last" : position;
+        var d = $("<li />"),
+          s = this._get_settings().core,
+          tmp;
+
+        if(obj !== -1 && !obj.length) { return false; }
+        if(!is_loaded && !this._is_loaded(obj)) { this.load_node(obj, function () { this.create_node(obj, position, js, callback, true); }); return false; }
+
+        this.__rollback();
+
+        if(typeof js === "string") { js = { "data" : js }; }
+        if(!js) { js = {}; }
+        if(js.attr) { d.attr(js.attr); }
+        if(js.metadata) { d.data(js.metadata); }
+        if(js.state) { d.addClass("jstree-" + js.state); }
+        if(!js.data) { js.data = this._get_string("new_node"); }
+        if(!$.isArray(js.data)) { tmp = js.data; js.data = []; js.data.push(tmp); }
+        $.each(js.data, function (i, m) {
+          tmp = $("<a />");
+          if($.isFunction(m)) { m = m.call(this, js); }
+          if(typeof m == "string") { tmp.attr('href','#')[ s.html_titles ? "html" : "text" ](m); }
+          else {
+            if(!m.attr) { m.attr = {}; }
+            if(!m.attr.href) { m.attr.href = '#'; }
+            tmp.attr(m.attr)[ s.html_titles ? "html" : "text" ](m.title);
+            if(m.language) { tmp.addClass(m.language); }
+          }
+          tmp.prepend("<ins class='jstree-icon'>&#160;</ins>");
+          if(m.icon) {
+            if(m.icon.indexOf("/") === -1) { tmp.children("ins").addClass(m.icon); }
+            else { tmp.children("ins").css("background","url('" + m.icon + "') center center no-repeat"); }
+          }
+          d.append(tmp);
+        });
+        d.prepend("<ins class='jstree-icon'>&#160;</ins>");
+        if(obj === -1) {
+          obj = this.get_container();
+          if(position === "before") { position = "first"; }
+          if(position === "after") { position = "last"; }
+        }
+        switch(position) {
+          case "before": obj.before(d); tmp = this._get_parent(obj); break;
+          case "after" : obj.after(d);  tmp = this._get_parent(obj); break;
+          case "inside":
+          case "first" :
+            if(!obj.children("ul").length) { obj.append("<ul />"); }
+            obj.children("ul").prepend(d);
+            tmp = obj;
+            break;
+          case "last":
+            if(!obj.children("ul").length) { obj.append("<ul />"); }
+            obj.children("ul").append(d);
+            tmp = obj;
+            break;
+          default:
+            if(!obj.children("ul").length) { obj.append("<ul />"); }
+            if(!position) { position = 0; }
+            tmp = obj.children("ul").children("li").eq(position);
+            if(tmp.length) { tmp.before(d); }
+            else { obj.children("ul").append(d); }
+            tmp = obj;
+            break;
+        }
+        if(tmp === -1 || tmp.get(0) === this.get_container().get(0)) { tmp = -1; }
+        this.clean_node(tmp);
+        this.__callback({ "obj" : d, "parent" : tmp });
+        if(callback) { callback.call(this, d); }
+        return d;
+      },
+      // Basic operations: rename (deal with text)
+      get_text  : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        var s = this._get_settings().core.html_titles;
+        obj = obj.children("a:eq(0)");
+        if(s) {
+          obj = obj.clone();
+          obj.children("INS").remove();
+          return obj.html();
+        }
+        else {
+          obj = obj.contents().filter(function() { return this.nodeType == 3; })[0];
+          return obj.nodeValue;
+        }
+      },
+      set_text  : function (obj, val) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        obj = obj.children("a:eq(0)");
+        if(this._get_settings().core.html_titles) {
+          var tmp = obj.children("INS").clone();
+          obj.html(val).prepend(tmp);
+          this.__callback({ "obj" : obj, "name" : val });
+          return true;
+        }
+        else {
+          obj = obj.contents().filter(function() { return this.nodeType == 3; })[0];
+          this.__callback({ "obj" : obj, "name" : val });
+          return (obj.nodeValue = val);
+        }
+      },
+      rename_node : function (obj, val) {
+        obj = this._get_node(obj);
+        this.__rollback();
+        if(obj && obj.length && this.set_text.apply(this, Array.prototype.slice.call(arguments))) { this.__callback({ "obj" : obj, "name" : val }); }
+      },
+      // Basic operations: deleting nodes
+      delete_node : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        this.__rollback();
+        var p = this._get_parent(obj), prev = $([]), t = this;
+        obj.each(function () {
+          prev = prev.add(t._get_prev(this));
+        });
+        obj = obj.detach();
+        if(p !== -1 && p.find("> ul > li").length === 0) {
+          p.removeClass("jstree-open jstree-closed").addClass("jstree-leaf");
+        }
+        this.clean_node(p);
+        this.__callback({ "obj" : obj, "prev" : prev, "parent" : p });
+        return obj;
+      },
+      prepare_move : function (o, r, pos, cb, is_cb) {
+        var p = {};
+
+        p.ot = $.jstree._reference(o) || this;
+        p.o = p.ot._get_node(o);
+        p.r = r === - 1 ? -1 : this._get_node(r);
+        p.p = (typeof pos === "undefined" || pos === false) ? "last" : pos; // TODO: move to a setting
+        if(!is_cb && prepared_move.o && prepared_move.o[0] === p.o[0] && prepared_move.r[0] === p.r[0] && prepared_move.p === p.p) {
+          this.__callback(prepared_move);
+          if(cb) { cb.call(this, prepared_move); }
+          return;
+        }
+        p.ot = $.jstree._reference(p.o) || this;
+        p.rt = $.jstree._reference(p.r) || this; // r === -1 ? p.ot : $.jstree._reference(p.r) || this
+        if(p.r === -1 || !p.r) {
+          p.cr = -1;
+          switch(p.p) {
+            case "first":
+            case "before":
+            case "inside":
+              p.cp = 0;
+              break;
+            case "after":
+            case "last":
+              p.cp = p.rt.get_container().find(" > ul > li").length;
+              break;
+            default:
+              p.cp = p.p;
+              break;
+          }
+        }
+        else {
+          if(!/^(before|after)$/.test(p.p) && !this._is_loaded(p.r)) {
+            return this.load_node(p.r, function () { this.prepare_move(o, r, pos, cb, true); });
+          }
+          switch(p.p) {
+            case "before":
+              p.cp = p.r.index();
+              p.cr = p.rt._get_parent(p.r);
+              break;
+            case "after":
+              p.cp = p.r.index() + 1;
+              p.cr = p.rt._get_parent(p.r);
+              break;
+            case "inside":
+            case "first":
+              p.cp = 0;
+              p.cr = p.r;
+              break;
+            case "last":
+              p.cp = p.r.find(" > ul > li").length;
+              p.cr = p.r;
+              break;
+            default:
+              p.cp = p.p;
+              p.cr = p.r;
+              break;
+          }
+        }
+        p.np = p.cr == -1 ? p.rt.get_container() : p.cr;
+        p.op = p.ot._get_parent(p.o);
+        p.cop = p.o.index();
+        if(p.op === -1) { p.op = p.ot ? p.ot.get_container() : this.get_container(); }
+        if(!/^(before|after)$/.test(p.p) && p.op && p.np && p.op[0] === p.np[0] && p.o.index() < p.cp) { p.cp++; }
+        //if(p.p === "before" && p.op && p.np && p.op[0] === p.np[0] && p.o.index() < p.cp) { p.cp--; }
+        p.or = p.np.find(" > ul > li:nth-child(" + (p.cp + 1) + ")");
+        prepared_move = p;
+        this.__callback(prepared_move);
+        if(cb) { cb.call(this, prepared_move); }
+      },
+      check_move : function () {
+        var obj = prepared_move, ret = true, r = obj.r === -1 ? this.get_container() : obj.r;
+        if(!obj || !obj.o || obj.or[0] === obj.o[0]) { return false; }
+        if(obj.op && obj.np && obj.op[0] === obj.np[0] && obj.cp - 1 === obj.o.index()) { return false; }
+        obj.o.each(function () {
+          if(r.parentsUntil(".jstree", "li").andSelf().index(this) !== -1) { ret = false; return false; }
+        });
+        return ret;
+      },
+      move_node : function (obj, ref, position, is_copy, is_prepared, skip_check) {
+        if(!is_prepared) {
+          return this.prepare_move(obj, ref, position, function (p) {
+            this.move_node(p, false, false, is_copy, true, skip_check);
+          });
+        }
+        if(is_copy) {
+          prepared_move.cy = true;
+        }
+        if(!skip_check && !this.check_move()) { return false; }
+
+        this.__rollback();
+        var o = false;
+        if(is_copy) {
+          o = obj.o.clone(true);
+          o.find("*[id]").andSelf().each(function () {
+            if(this.id) { this.id = "copy_" + this.id; }
+          });
+        }
+        else { o = obj.o; }
+
+        if(obj.or.length) { obj.or.before(o); }
+        else {
+          if(!obj.np.children("ul").length) { $("<ul />").appendTo(obj.np); }
+          obj.np.children("ul:eq(0)").append(o);
+        }
+
+        try {
+          obj.ot.clean_node(obj.op);
+          obj.rt.clean_node(obj.np);
+          if(!obj.op.find("> ul > li").length) {
+            obj.op.removeClass("jstree-open jstree-closed").addClass("jstree-leaf").children("ul").remove();
+          }
+        } catch (e) { }
+
+        if(is_copy) {
+          prepared_move.cy = true;
+          prepared_move.oc = o;
+        }
+        this.__callback(prepared_move);
+        return prepared_move;
+      },
+      _get_move : function () { return prepared_move; }
+    }
+  });
+})(jQuery);
+//*/
+
+/*
+ * jsTree ui plugin
+ * This plugins handles selecting/deselecting/hovering/dehovering nodes
+ */
+(function ($) {
+  var scrollbar_width, e1, e2;
+  $(function() {
+    if (/msie/.test(navigator.userAgent.toLowerCase())) {
+      e1 = $('<textarea cols="10" rows="2"></textarea>').css({ position: 'absolute', top: -1000, left: 0 }).appendTo('body');
+      e2 = $('<textarea cols="10" rows="2" style="overflow: hidden;"></textarea>').css({ position: 'absolute', top: -1000, left: 0 }).appendTo('body');
+      scrollbar_width = e1.width() - e2.width();
+      e1.add(e2).remove();
+    }
+    else {
+      e1 = $('<div />').css({ width: 100, height: 100, overflow: 'auto', position: 'absolute', top: -1000, left: 0 })
+          .prependTo('body').append('<div />').find('div').css({ width: '100%', height: 200 });
+      scrollbar_width = 100 - e1.width();
+      e1.parent().remove();
+    }
+  });
+  $.jstree.plugin("ui", {
+    __init : function () {
+      this.data.ui.selected = $();
+      this.data.ui.last_selected = false;
+      this.data.ui.hovered = null;
+      this.data.ui.to_select = this.get_settings().ui.initially_select;
+
+      this.get_container()
+        .delegate("a", "click.jstree", $.proxy(function (event) {
+            event.preventDefault();
+            event.currentTarget.blur();
+            if(!$(event.currentTarget).hasClass("jstree-loading")) {
+              this.select_node(event.currentTarget, true, event);
+            }
+          }, this))
+        .delegate("a", "mouseenter.jstree", $.proxy(function (event) {
+            if(!$(event.currentTarget).hasClass("jstree-loading")) {
+              this.hover_node(event.target);
+            }
+          }, this))
+        .delegate("a", "mouseleave.jstree", $.proxy(function (event) {
+            if(!$(event.currentTarget).hasClass("jstree-loading")) {
+              this.dehover_node(event.target);
+            }
+          }, this))
+        .bind("reopen.jstree", $.proxy(function () {
+            this.reselect();
+          }, this))
+        .bind("get_rollback.jstree", $.proxy(function () {
+            this.dehover_node();
+            this.save_selected();
+          }, this))
+        .bind("set_rollback.jstree", $.proxy(function () {
+            this.reselect();
+          }, this))
+        .bind("close_node.jstree", $.proxy(function (event, data) {
+            var s = this._get_settings().ui,
+              obj = this._get_node(data.rslt.obj),
+              clk = (obj && obj.length) ? obj.children("ul").find("a.jstree-clicked") : $(),
+              _this = this;
+            if(s.selected_parent_close === false || !clk.length) { return; }
+            clk.each(function () {
+              _this.deselect_node(this);
+              if(s.selected_parent_close === "select_parent") { _this.select_node(obj); }
+            });
+          }, this))
+        .bind("delete_node.jstree", $.proxy(function (event, data) {
+            var s = this._get_settings().ui.select_prev_on_delete,
+              obj = this._get_node(data.rslt.obj),
+              clk = (obj && obj.length) ? obj.find("a.jstree-clicked") : [],
+              _this = this;
+            clk.each(function () { _this.deselect_node(this); });
+            if(s && clk.length) {
+              data.rslt.prev.each(function () {
+                if(this.parentNode) { _this.select_node(this); return false; /* if return false is removed all prev nodes will be selected */}
+              });
+            }
+          }, this))
+        .bind("move_node.jstree", $.proxy(function (event, data) {
+            if(data.rslt.cy) {
+              data.rslt.oc.find("a.jstree-clicked").removeClass("jstree-clicked");
+            }
+          }, this));
+    },
+    defaults : {
+      select_limit : -1, // 0, 1, 2 ... or -1 for unlimited
+      select_multiple_modifier : "ctrl", // on, or ctrl, shift, alt
+      select_range_modifier : "shift",
+      selected_parent_close : "select_parent", // false, "deselect", "select_parent"
+      selected_parent_open : true,
+      select_prev_on_delete : true,
+      disable_selecting_children : false,
+      initially_select : []
+    },
+    _fn : {
+      _get_node : function (obj, allow_multiple) {
+        if(typeof obj === "undefined" || obj === null) { return allow_multiple ? this.data.ui.selected : this.data.ui.last_selected; }
+        var $obj = $(obj, this.get_container());
+        if($obj.is(".jstree") || obj == -1) { return -1; }
+        $obj = $obj.closest("li", this.get_container());
+        return $obj.length ? $obj : false;
+      },
+      _ui_notify : function (n, data) {
+        if(data.selected) {
+          this.select_node(n, false);
+        }
+      },
+      save_selected : function () {
+        var _this = this;
+        this.data.ui.to_select = [];
+        this.data.ui.selected.each(function () { if(this.id) { _this.data.ui.to_select.push("#" + this.id.toString().replace(/^#/,"").replace(/\\\//g,"/").replace(/\//g,"\\\/").replace(/\\\./g,".").replace(/\./g,"\\.").replace(/\:/g,"\\:")); } });
+        this.__callback(this.data.ui.to_select);
+      },
+      reselect : function () {
+        var _this = this,
+          s = this.data.ui.to_select;
+        s = $.map($.makeArray(s), function (n) { return "#" + n.toString().replace(/^#/,"").replace(/\\\//g,"/").replace(/\//g,"\\\/").replace(/\\\./g,".").replace(/\./g,"\\.").replace(/\:/g,"\\:"); });
+        // this.deselect_all(); WHY deselect, breaks plugin state notifier?
+        $.each(s, function (i, val) { if(val && val !== "#") { _this.select_node(val); } });
+        this.data.ui.selected = this.data.ui.selected.filter(function () { return this.parentNode; });
+        this.__callback();
+      },
+      refresh : function (obj) {
+        this.save_selected();
+        return this.__call_old();
+      },
+      hover_node : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        //if(this.data.ui.hovered && obj.get(0) === this.data.ui.hovered.get(0)) { return; }
+        if(!obj.hasClass("jstree-hovered")) { this.dehover_node(); }
+        this.data.ui.hovered = obj.children("a").addClass("jstree-hovered").parent();
+        this._fix_scroll(obj);
+        this.__callback({ "obj" : obj });
+      },
+      dehover_node : function () {
+        var obj = this.data.ui.hovered, p;
+        if(!obj || !obj.length) { return false; }
+        p = obj.children("a").removeClass("jstree-hovered").parent();
+        if(this.data.ui.hovered[0] === p[0]) { this.data.ui.hovered = null; }
+        this.__callback({ "obj" : obj });
+      },
+      select_node : function (obj, check, e) {
+        obj = this._get_node(obj);
+        if(obj == -1 || !obj || !obj.length) { return false; }
+        var s = this._get_settings().ui,
+          is_multiple = (s.select_multiple_modifier == "on" || (s.select_multiple_modifier !== false && e && e[s.select_multiple_modifier + "Key"])),
+          is_range = (s.select_range_modifier !== false && e && e[s.select_range_modifier + "Key"] && this.data.ui.last_selected && this.data.ui.last_selected[0] !== obj[0] && this.data.ui.last_selected.parent()[0] === obj.parent()[0]),
+          is_selected = this.is_selected(obj),
+          proceed = true,
+          t = this;
+        if(check) {
+          if(s.disable_selecting_children && is_multiple &&
+            (
+              (obj.parentsUntil(".jstree","li").children("a.jstree-clicked").length) ||
+              (obj.children("ul").find("a.jstree-clicked:eq(0)").length)
+            )
+          ) {
+            return false;
+          }
+          proceed = false;
+          switch(!0) {
+            case (is_range):
+              this.data.ui.last_selected.addClass("jstree-last-selected");
+              obj = obj[ obj.index() < this.data.ui.last_selected.index() ? "nextUntil" : "prevUntil" ](".jstree-last-selected").andSelf();
+              if(s.select_limit == -1 || obj.length < s.select_limit) {
+                this.data.ui.last_selected.removeClass("jstree-last-selected");
+                this.data.ui.selected.each(function () {
+                  if(this !== t.data.ui.last_selected[0]) { t.deselect_node(this); }
+                });
+                is_selected = false;
+                proceed = true;
+              }
+              else {
+                proceed = false;
+              }
+              break;
+            case (is_selected && !is_multiple):
+              this.deselect_all();
+              is_selected = false;
+              proceed = true;
+              break;
+            case (!is_selected && !is_multiple):
+              if(s.select_limit == -1 || s.select_limit > 0) {
+                this.deselect_all();
+                proceed = true;
+              }
+              break;
+            case (is_selected && is_multiple):
+              this.deselect_node(obj);
+              break;
+            case (!is_selected && is_multiple):
+              if(s.select_limit == -1 || this.data.ui.selected.length + 1 <= s.select_limit) {
+                proceed = true;
+              }
+              break;
+          }
+        }
+        if(proceed && !is_selected) {
+          if(!is_range) { this.data.ui.last_selected = obj; }
+          obj.children("a").addClass("jstree-clicked");
+          if(s.selected_parent_open) {
+            obj.parents(".jstree-closed").each(function () { t.open_node(this, false, true); });
+          }
+          this.data.ui.selected = this.data.ui.selected.add(obj);
+          this._fix_scroll(obj.eq(0));
+          this.__callback({ "obj" : obj, "e" : e });
+        }
+      },
+      _fix_scroll : function (obj) {
+        var c = this.get_container()[0], t;
+        if(c.scrollHeight > c.offsetHeight) {
+          obj = this._get_node(obj);
+          if(!obj || obj === -1 || !obj.length || !obj.is(":visible")) { return; }
+          t = obj.offset().top - this.get_container().offset().top;
+          if(t < 0) {
+            c.scrollTop = c.scrollTop + t - 1;
+          }
+          if(t + this.data.core.li_height + (c.scrollWidth > c.offsetWidth ? scrollbar_width : 0) > c.offsetHeight) {
+            c.scrollTop = c.scrollTop + (t - c.offsetHeight + this.data.core.li_height + 1 + (c.scrollWidth > c.offsetWidth ? scrollbar_width : 0));
+          }
+        }
+      },
+      deselect_node : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        if(this.is_selected(obj)) {
+          obj.children("a").removeClass("jstree-clicked");
+          this.data.ui.selected = this.data.ui.selected.not(obj);
+          if(this.data.ui.last_selected.get(0) === obj.get(0)) { this.data.ui.last_selected = this.data.ui.selected.eq(0); }
+          this.__callback({ "obj" : obj });
+        }
+      },
+      toggle_select : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj.length) { return false; }
+        if(this.is_selected(obj)) { this.deselect_node(obj); }
+        else { this.select_node(obj); }
+      },
+      is_selected : function (obj) { return this.data.ui.selected.index(this._get_node(obj)) >= 0; },
+      get_selected : function (context) {
+        return context ? $(context).find("a.jstree-clicked").parent() : this.data.ui.selected;
+      },
+      deselect_all : function (context) {
+        var ret = context ? $(context).find("a.jstree-clicked").parent() : this.get_container().find("a.jstree-clicked").parent();
+        ret.children("a.jstree-clicked").removeClass("jstree-clicked");
+        this.data.ui.selected = $([]);
+        this.data.ui.last_selected = false;
+        this.__callback({ "obj" : ret });
+      }
+    }
+  });
+  // include the selection plugin by default
+  $.jstree.defaults.plugins.push("ui");
+})(jQuery);
+//*/
+
+/*
+ * jsTree CRRM plugin
+ * Handles creating/renaming/removing/moving nodes by user interaction.
+ */
+(function ($) {
+  $.jstree.plugin("crrm", {
+    __init : function () {
+      this.get_container()
+        .bind("move_node.jstree", $.proxy(function (e, data) {
+          if(this._get_settings().crrm.move.open_onmove) {
+            var t = this;
+            data.rslt.np.parentsUntil(".jstree").andSelf().filter(".jstree-closed").each(function () {
+              t.open_node(this, false, true);
+            });
+          }
+        }, this));
+    },
+    defaults : {
+      input_width_limit : 200,
+      move : {
+        always_copy      : false, // false, true or "multitree"
+        open_onmove      : true,
+        default_position  : "last",
+        check_move      : function (m) { return true; }
+      }
+    },
+    _fn : {
+      _show_input : function (obj, callback) {
+        obj = this._get_node(obj);
+        var rtl = this._get_settings().core.rtl,
+          w = this._get_settings().crrm.input_width_limit,
+          w1 = obj.children("ins").width(),
+          w2 = obj.find("> a:visible > ins").width() * obj.find("> a:visible > ins").length,
+          t = this.get_text(obj),
+          h1 = $("<div />", { css : { "position" : "absolute", "top" : "-200px", "left" : (rtl ? "0px" : "-1000px"), "visibility" : "hidden" } }).appendTo("body"),
+          h2 = obj.css("position","relative").append(
+          $("<input />", {
+            "value" : t,
+            "class" : "jstree-rename-input",
+            // "size" : t.length,
+            "css" : {
+              "padding" : "0",
+              "border" : "1px solid silver",
+              "position" : "absolute",
+              "left"  : (rtl ? "auto" : (w1 + w2 + 4) + "px"),
+              "right" : (rtl ? (w1 + w2 + 4) + "px" : "auto"),
+              "top" : "0px",
+              "height" : (this.data.core.li_height - 2) + "px",
+              "lineHeight" : (this.data.core.li_height - 2) + "px",
+              "width" : "150px" // will be set a bit further down
+            },
+            "blur" : $.proxy(function () {
+              var i = obj.children(".jstree-rename-input"),
+                v = i.val();
+              if(v === "") { v = t; }
+              h1.remove();
+              i.remove(); // rollback purposes
+              this.set_text(obj,t); // rollback purposes
+              this.rename_node(obj, v);
+              callback.call(this, obj, v, t);
+              obj.css("position","");
+            }, this),
+            "keyup" : function (event) {
+              var key = event.keyCode || event.which;
+              if(key == 27) { this.value = t; this.blur(); return; }
+              else if(key == 13) { this.blur(); return; }
+              else {
+                h2.width(Math.min(h1.text("pW" + this.value).width(),w));
+              }
+            },
+            "keypress" : function(event) {
+              var key = event.keyCode || event.which;
+              if(key == 13) { return false; }
+            }
+          })
+        ).children(".jstree-rename-input");
+        this.set_text(obj, "");
+        h1.css({
+            fontFamily    : h2.css('fontFamily')    || '',
+            fontSize    : h2.css('fontSize')    || '',
+            fontWeight    : h2.css('fontWeight')    || '',
+            fontStyle    : h2.css('fontStyle')    || '',
+            fontStretch    : h2.css('fontStretch')    || '',
+            fontVariant    : h2.css('fontVariant')    || '',
+            letterSpacing  : h2.css('letterSpacing')  || '',
+            wordSpacing    : h2.css('wordSpacing')    || ''
+        });
+        h2.width(Math.min(h1.text("pW" + h2[0].value).width(),w))[0].select();
+      },
+      rename : function (obj) {
+        obj = this._get_node(obj);
+        this.__rollback();
+        var f = this.__callback;
+        this._show_input(obj, function (obj, new_name, old_name) {
+          f.call(this, { "obj" : obj, "new_name" : new_name, "old_name" : old_name });
+        });
+      },
+      create : function (obj, position, js, callback, skip_rename) {
+        var t, _this = this;
+        obj = this._get_node(obj);
+        if(!obj) { obj = -1; }
+        this.__rollback();
+        t = this.create_node(obj, position, js, function (t) {
+          var p = this._get_parent(t),
+            pos = $(t).index();
+          if(callback) { callback.call(this, t); }
+          if(p.length && p.hasClass("jstree-closed")) { this.open_node(p, false, true); }
+          if(!skip_rename) {
+            this._show_input(t, function (obj, new_name, old_name) {
+              _this.__callback({ "obj" : obj, "name" : new_name, "parent" : p, "position" : pos });
+            });
+          }
+          else { _this.__callback({ "obj" : t, "name" : this.get_text(t), "parent" : p, "position" : pos }); }
+        });
+        return t;
+      },
+      remove : function (obj) {
+        obj = this._get_node(obj, true);
+        var p = this._get_parent(obj), prev = this._get_prev(obj);
+        this.__rollback();
+        obj = this.delete_node(obj);
+        if(obj !== false) { this.__callback({ "obj" : obj, "prev" : prev, "parent" : p }); }
+      },
+      check_move : function () {
+        if(!this.__call_old()) { return false; }
+        var s = this._get_settings().crrm.move;
+        if(!s.check_move.call(this, this._get_move())) { return false; }
+        return true;
+      },
+      move_node : function (obj, ref, position, is_copy, is_prepared, skip_check) {
+        var s = this._get_settings().crrm.move;
+        if(!is_prepared) {
+          if(typeof position === "undefined") { position = s.default_position; }
+          if(position === "inside" && !s.default_position.match(/^(before|after)$/)) { position = s.default_position; }
+          return this.__call_old(true, obj, ref, position, is_copy, false, skip_check);
+        }
+        // if the move is already prepared
+        if(s.always_copy === true || (s.always_copy === "multitree" && obj.rt.get_index() !== obj.ot.get_index() )) {
+          is_copy = true;
+        }
+        this.__call_old(true, obj, ref, position, is_copy, true, skip_check);
+      },
+
+      cut : function (obj) {
+        obj = this._get_node(obj, true);
+        if(!obj || !obj.length) { return false; }
+        this.data.crrm.cp_nodes = false;
+        this.data.crrm.ct_nodes = obj;
+        this.__callback({ "obj" : obj });
+      },
+      copy : function (obj) {
+        obj = this._get_node(obj, true);
+        if(!obj || !obj.length) { return false; }
+        this.data.crrm.ct_nodes = false;
+        this.data.crrm.cp_nodes = obj;
+        this.__callback({ "obj" : obj });
+      },
+      paste : function (obj) {
+        obj = this._get_node(obj);
+        if(!obj || !obj.length) { return false; }
+        var nodes = this.data.crrm.ct_nodes ? this.data.crrm.ct_nodes : this.data.crrm.cp_nodes;
+        if(!this.data.crrm.ct_nodes && !this.data.crrm.cp_nodes) { return false; }
+        if(this.data.crrm.ct_nodes) { this.move_node(this.data.crrm.ct_nodes, obj); this.data.crrm.ct_nodes = false; }
+        if(this.data.crrm.cp_nodes) { this.move_node(this.data.crrm.cp_nodes, obj, false, true); }
+        this.__callback({ "obj" : obj, "nodes" : nodes });
+      }
+    }
+  });
+  // include the crr plugin by default
+  // $.jstree.defaults.plugins.push("crrm");
+})(jQuery);
+//*/
+
+/*
+ * jsTree themes plugin
+ * Handles loading and setting themes, as well as detecting path to themes, etc.
+ */
+(function ($) {
+  var themes_loaded = [];
+  // this variable stores the path to the themes folder - if left as false - it will be autodetected
+  $.jstree._themes = false;
+  $.jstree.plugin("themes", {
+    __init : function () {
+      this.get_container()
+        .bind("init.jstree", $.proxy(function () {
+            var s = this._get_settings().themes;
+            this.data.themes.dots = s.dots;
+            this.data.themes.icons = s.icons;
+            this.set_theme(s.theme, s.url);
+          }, this))
+        .bind("loaded.jstree", $.proxy(function () {
+            // bound here too, as simple HTML tree's won't honor dots & icons otherwise
+            if(!this.data.themes.dots) { this.hide_dots(); }
+            else { this.show_dots(); }
+            if(!this.data.themes.icons) { this.hide_icons(); }
+            else { this.show_icons(); }
+          }, this));
+    },
+    defaults : {
+      theme : "default",
+      url : false,
+      dots : true,
+      icons : true
+    },
+    _fn : {
+      set_theme : function (theme_name, theme_url) {
+        if(!theme_name) { return false; }
+        if(!theme_url) { theme_url = $.jstree._themes + theme_name + '/style.css'; }
+        if($.inArray(theme_url, themes_loaded) == -1) {
+          $.vakata.css.add_sheet({ "url" : theme_url });
+          themes_loaded.push(theme_url);
+        }
+        if(this.data.themes.theme != theme_name) {
+          this.get_container().removeClass('jstree-' + this.data.themes.theme);
+          this.data.themes.theme = theme_name;
+        }
+        this.get_container().addClass('jstree-' + theme_name);
+        if(!this.data.themes.dots) { this.hide_dots(); }
+        else { this.show_dots(); }
+        if(!this.data.themes.icons) { this.hide_icons(); }
+        else { this.show_icons(); }
+        this.__callback();
+      },
+      get_theme  : function () { return this.data.themes.theme; },
+
+      show_dots  : function () { this.data.themes.dots = true; this.get_container().children("ul").removeClass("jstree-no-dots"); },
+      hide_dots  : function () { this.data.themes.dots = false; this.get_container().children("ul").addClass("jstree-no-dots"); },
+      toggle_dots  : function () { if(this.data.themes.dots) { this.hide_dots(); } else { this.show_dots(); } },
+
+      show_icons  : function () { this.data.themes.icons = true; this.get_container().children("ul").removeClass("jstree-no-icons"); },
+      hide_icons  : function () { this.data.themes.icons = false; this.get_container().children("ul").addClass("jstree-no-icons"); },
+      toggle_icons: function () { if(this.data.themes.icons) { this.hide_icons(); } else { this.show_icons(); } }
+    }
+  });
+  // autodetect themes path
+  $(function () {
+    if($.jstree._themes === false) {
+      $("script").each(function () {
+        if(this.src.toString().match(/jquery\.jstree[^\/]*?\.js(\?.*)?$/)) {
+          $.jstree._themes = this.src.toString().replace(/jquery\.jstree[^\/]*?\.js(\?.*)?$/, "") + 'themes/';
+          return false;
+        }
+      });
+    }
+    if($.jstree._themes === false) { $.jstree._themes = "themes/"; }
+  });
+  // include the themes plugin by default
+  $.jstree.defaults.plugins.push("themes");
+})(jQuery);
+//*/
+
+/*
+ * jsTree hotkeys plugin
+ * Enables keyboard navigation for all tree instances
+ * Depends on the jstree ui & jquery hotkeys plugins
+ */
+(function ($) {
+  var bound = [];
+  function exec(i, event) {
+    var f = $.jstree._focused(), tmp;
+    if(f && f.data && f.data.hotkeys && f.data.hotkeys.enabled) {
+      tmp = f._get_settings().hotkeys[i];
+      if(tmp) { return tmp.call(f, event); }
+    }
+  }
+  $.jstree.plugin("hotkeys", {
+    __init : function () {
+      if(typeof $.hotkeys === "undefined") { throw "jsTree hotkeys: jQuery hotkeys plugin not included."; }
+      if(!this.data.ui) { throw "jsTree hotkeys: jsTree UI plugin not included."; }
+      $.each(this._get_settings().hotkeys, function (i, v) {
+        if(v !== false && $.inArray(i, bound) == -1) {
+          $(document).bind("keydown", i, function (event) { return exec(i, event); });
+          bound.push(i);
+        }
+      });
+      this.get_container()
+        .bind("lock.jstree", $.proxy(function () {
+            if(this.data.hotkeys.enabled) { this.data.hotkeys.enabled = false; this.data.hotkeys.revert = true; }
+          }, this))
+        .bind("unlock.jstree", $.proxy(function () {
+            if(this.data.hotkeys.revert) { this.data.hotkeys.enabled = true; }
+          }, this));
+      this.enable_hotkeys();
+    },
+    defaults : {
+      "up" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected || -1;
+        this.hover_node(this._get_prev(o));
+        return false;
+      },
+      "ctrl+up" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected || -1;
+        this.hover_node(this._get_prev(o));
+        return false;
+      },
+      "shift+up" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected || -1;
+        this.hover_node(this._get_prev(o));
+        return false;
+      },
+      "down" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected || -1;
+        this.hover_node(this._get_next(o));
+        return false;
+      },
+      "ctrl+down" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected || -1;
+        this.hover_node(this._get_next(o));
+        return false;
+      },
+      "shift+down" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected || -1;
+        this.hover_node(this._get_next(o));
+        return false;
+      },
+      "left" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected;
+        if(o) {
+          if(o.hasClass("jstree-open")) { this.close_node(o); }
+          else { this.hover_node(this._get_prev(o)); }
+        }
+        return false;
+      },
+      "ctrl+left" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected;
+        if(o) {
+          if(o.hasClass("jstree-open")) { this.close_node(o); }
+          else { this.hover_node(this._get_prev(o)); }
+        }
+        return false;
+      },
+      "shift+left" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected;
+        if(o) {
+          if(o.hasClass("jstree-open")) { this.close_node(o); }
+          else { this.hover_node(this._get_prev(o)); }
+        }
+        return false;
+      },
+      "right" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected;
+        if(o && o.length) {
+          if(o.hasClass("jstree-closed")) { this.open_node(o); }
+          else { this.hover_node(this._get_next(o)); }
+        }
+        return false;
+      },
+      "ctrl+right" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected;
+        if(o && o.length) {
+          if(o.hasClass("jstree-closed")) { this.open_node(o); }
+          else { this.hover_node(this._get_next(o)); }
+        }
+        return false;
+      },
+      "shift+right" : function () {
+        var o = this.data.ui.hovered || this.data.ui.last_selected;
+        if(o && o.length) {
+          if(o.hasClass("jstree-closed")) { this.open_node(o); }
+          else { this.hover_node(this._get_next(o)); }
+        }
+        return false;
+      },
+      "space" : function () {
+        if(this.data.ui.hovered) { this.data.ui.hovered.children("a:eq(0)").click(); }
+        return false;
+      },
+      "ctrl+space" : function (event) {
+        event.type = "click";
+        if(this.data.ui.hovered) { this.data.ui.hovered.children("a:eq(0)").trigger(event); }
+        return false;
+      },
+      "shift+space" : function (event) {
+        event.type = "click";
+        if(this.data.ui.hovered) { this.data.ui.hovered.children("a:eq(0)").trigger(event); }
+        return false;
+      },
+      "f2" : function () { this.rename(this.data.ui.hovered || this.data.ui.last_selected); },
+      "del" : function () { this.remove(this.data.ui.hovered || this._get_node(null)); }
+    },
+    _fn : {
+      enable_hotkeys : function () {
+        this.data.hotkeys.enabled = true;
+      },
+      disable_hotkeys : function () {
+        this.data.hotkeys.enabled = false;
+      }
+    }
+  });
+})(jQuery);
+//*/
+
+/*
+ * jsTree JSON plugin
+ * The JSON data store. Datastores are build by overriding the `load_node` and `_is_loaded` functions.
+ */
+(function ($) {
+  $.jstree.plugin("json_data", {
+    __init : function() {
+      var s = this._get_settings().json_data;
+      if(s.progressive_unload) {
+        this.get_container().bind("after_close.jstree", function (e, data) {
+          data.rslt.obj.children("ul").remove();
+        });
+      }
+    },
+    defaults : {
+      // `data` can be a function:
+      //  * accepts two arguments - node being loaded and a callback to pass the result to
+      //  * will be executed in the current tree's scope & ajax won't be supported
+      data : false,
+      ajax : false,
+      correct_state : true,
+      progressive_render : false,
+      progressive_unload : false
+    },
+    _fn : {
+      load_node : function (obj, s_call, e_call) { var _this = this; this.load_node_json(obj, function () { _this.__callback({ "obj" : _this._get_node(obj) }); s_call.call(this); }, e_call); },
+      _is_loaded : function (obj) {
+        var s = this._get_settings().json_data;
+        obj = this._get_node(obj);
+        return obj == -1 || !obj || (!s.ajax && !s.progressive_render && !$.isFunction(s.data)) || obj.is(".jstree-open, .jstree-leaf") || obj.children("ul").children("li").length > 0;
+      },
+      refresh : function (obj) {
+        obj = this._get_node(obj);
+        var s = this._get_settings().json_data;
+        if(obj && obj !== -1 && s.progressive_unload && ($.isFunction(s.data) || !!s.ajax)) {
+          obj.removeData("jstree-children");
+        }
+        return this.__call_old();
+      },
+      load_node_json : function (obj, s_call, e_call) {
+        var s = this.get_settings().json_data, d,
+          error_func = function () {},
+          success_func = function () {};
+        obj = this._get_node(obj);
+
+        if(obj && obj !== -1 && (s.progressive_render || s.progressive_unload) && !obj.is(".jstree-open, .jstree-leaf") && obj.children("ul").children("li").length === 0 && obj.data("jstree-children")) {
+          d = this._parse_json(obj.data("jstree-children"), obj);
+          if(d) {
+            obj.append(d);
+            if(!s.progressive_unload) { obj.removeData("jstree-children"); }
+          }
+          this.clean_node(obj);
+          if(s_call) { s_call.call(this); }
+          return;
+        }
+
+        if(obj && obj !== -1) {
+          if(obj.data("jstree-is-loading")) { return; }
+          else { obj.data("jstree-is-loading",true); }
+        }
+        switch(!0) {
+          case (!s.data && !s.ajax): throw "Neither data nor ajax settings supplied.";
+          // function option added here for easier model integration (also supporting async - see callback)
+          case ($.isFunction(s.data)):
+            s.data.call(this, obj, $.proxy(function (d) {
+              d = this._parse_json(d, obj);
+              if(!d) {
+                if(obj === -1 || !obj) {
+                  if(s.correct_state) { this.get_container().children("ul").empty(); }
+                }
+                else {
+                  obj.children("a.jstree-loading").removeClass("jstree-loading");
+                  obj.removeData("jstree-is-loading");
+                  if(s.correct_state) { this.correct_state(obj); }
+                }
+                if(e_call) { e_call.call(this); }
+              }
+              else {
+                if(obj === -1 || !obj) { this.get_container().children("ul").empty().append(d.children()); }
+                else { obj.append(d).children("a.jstree-loading").removeClass("jstree-loading"); obj.removeData("jstree-is-loading"); }
+                this.clean_node(obj);
+                if(s_call) { s_call.call(this); }
+              }
+            }, this));
+            break;
+          case (!!s.data && !s.ajax) || (!!s.data && !!s.ajax && (!obj || obj === -1)):
+            if(!obj || obj == -1) {
+              d = this._parse_json(s.data, obj);
+              if(d) {
+                this.get_container().children("ul").empty().append(d.children());
+                this.clean_node();
+              }
+              else {
+                if(s.correct_state) { this.get_container().children("ul").empty(); }
+              }
+            }
+            if(s_call) { s_call.call(this); }
+            break;
+          case (!s.data && !!s.ajax) || (!!s.data && !!s.ajax && obj && obj !== -1):
+            error_func = function (x, t, e) {
+              var ef = this.get_settings().json_data.ajax.error;
+              if(ef) { ef.call(this, x, t, e); }
+              if(obj != -1 && obj.length) {
+                obj.children("a.jstree-loading").removeClass("jstree-loading");
+                obj.removeData("jstree-is-loading");
+                if(t === "success" && s.correct_state) { this.correct_state(obj); }
+              }
+              else {
+                if(t === "success" && s.correct_state) { this.get_container().children("ul").empty(); }
+              }
+              if(e_call) { e_call.call(this); }
+            };
+            success_func = function (d, t, x) {
+              var sf = this.get_settings().json_data.ajax.success;
+              if(sf) { d = sf.call(this,d,t,x) || d; }
+              if(d === "" || (d && d.toString && d.toString().replace(/^[\s\n]+$/,"") === "") || (!$.isArray(d) && !$.isPlainObject(d))) {
+                return error_func.call(this, x, t, "");
+              }
+              d = this._parse_json(d, obj);
+              if(d) {
+                if(obj === -1 || !obj) { this.get_container().children("ul").empty().append(d.children()); }
+                else { obj.append(d).children("a.jstree-loading").removeClass("jstree-loading"); obj.removeData("jstree-is-loading"); }
+                this.clean_node(obj);
+                if(s_call) { s_call.call(this); }
+              }
+              else {
+                if(obj === -1 || !obj) {
+                  if(s.correct_state) {
+                    this.get_container().children("ul").empty();
+                    if(s_call) { s_call.call(this); }
+                  }
+                }
+                else {
+                  obj.children("a.jstree-loading").removeClass("jstree-loading");
+                  obj.removeData("jstree-is-loading");
+                  if(s.correct_state) {
+                    this.correct_state(obj);
+                    if(s_call) { s_call.call(this); }
+                  }
+                }
+              }
+            };
+            s.ajax.context = this;
+            s.ajax.error = error_func;
+            s.ajax.success = success_func;
+            if(!s.ajax.dataType) { s.ajax.dataType = "json"; }
+            if($.isFunction(s.ajax.url)) { s.ajax.url = s.ajax.url.call(this, obj); }
+            if($.isFunction(s.ajax.data)) { s.ajax.data = s.ajax.data.call(this, obj); }
+            $.ajax(s.ajax);
+            break;
+        }
+      },
+      _parse_json : function (js, obj, is_callback) {
+        var d = false,
+          p = this._get_settings(),
+          s = p.json_data,
+          t = p.core.html_titles,
+          tmp, i, j, ul1, ul2;
+
+        if(!js) { return d; }
+        if(s.progressive_unload && obj && obj !== -1) {
+          obj.data("jstree-children", d);
+        }
+        if($.isArray(js)) {
+          d = $();
+          if(!js.length) { return false; }
+          for(i = 0, j = js.length; i < j; i++) {
+            tmp = this._parse_json(js[i], obj, true);
+            if(tmp.length) { d = d.add(tmp); }
+          }
+        }
+        else {
+          if(typeof js == "string") { js = { data : js }; }
+          if(!js.data && js.data !== "") { return d; }
+          d = $("<li />");
+          if(js.attr) { d.attr(js.attr); }
+          if(js.metadata) { d.data(js.metadata); }
+          if(js.state) { d.addClass("jstree-" + js.state); }
+          if(!$.isArray(js.data)) { tmp = js.data; js.data = []; js.data.push(tmp); }
+          $.each(js.data, function (i, m) {
+            tmp = $("<a />");
+            if($.isFunction(m)) { m = m.call(this, js); }
+            if(typeof m == "string") { tmp.attr('href','#')[ t ? "html" : "text" ](m); }
+            else {
+              if(!m.attr) { m.attr = {}; }
+              if(!m.attr.href) { m.attr.href = '#'; }
+              tmp.attr(m.attr)[ t ? "html" : "text" ](m.title);
+              if(m.language) { tmp.addClass(m.language); }
+            }
+            tmp.prepend("<ins class='jstree-icon'>&#160;</ins>");
+            if(!m.icon && js.icon) { m.icon = js.icon; }
+            if(m.icon) {
+              if(m.icon.indexOf("/") === -1) { tmp.children("ins").addClass(m.icon); }
+              else { tmp.children("ins").css("background","url('" + m.icon + "') center center no-repeat"); }
+            }
+            d.append(tmp);
+          });
+          d.prepend("<ins class='jstree-icon'>&#160;</ins>");
+          if(js.children) {
+            if(s.progressive_render && js.state !== "open") {
+              d.addClass("jstree-closed").data("jstree-children", js.children);
+            }
+            else {
+              if(s.progressive_unload) { d.data("jstree-children", js.children); }
+              if($.isArray(js.children) && js.children.length) {
+                tmp = this._parse_json(js.children, obj, true);
+                if(tmp.length) {
+                  ul2 = $("<ul />");
+                  ul2.append(tmp);
+                  d.append(ul2);
+                }
+              }
+            }
+          }
+        }
+        if(!is_callback) {
+          ul1 = $("<ul />");
+          ul1.append(d);
+          d = ul1;
+        }
+        return d;
+      },
+      get_json : function (obj, li_attr, a_attr, is_callback) {
+        var result = [],
+          s = this._get_settings(),
+          _this = this,
+          tmp1, tmp2, li, a, t, lang;
+        obj = this._get_node(obj);
+        if(!obj || obj === -1) { obj = this.get_container().find("> ul > li"); }
+        li_attr = $.isArray(li_attr) ? li_attr : [ "id", "class" ];
+        if(!is_callback && this.data.types) { li_attr.push(s.types.type_attr); }
+        a_attr = $.isArray(a_attr) ? a_attr : [ ];
+
+        obj.each(function () {
+          li = $(this);
+          tmp1 = { data : [] };
+          if(li_attr.length) { tmp1.attr = { }; }
+          $.each(li_attr, function (i, v) {
+            tmp2 = li.attr(v);
+            if(tmp2 && tmp2.length && tmp2.replace(/jstree[^ ]*/ig,'').length) {
+              tmp1.attr[v] = (" " + tmp2).replace(/ jstree[^ ]*/ig,'').replace(/\s+$/ig," ").replace(/^ /,"").replace(/ $/,"");
+            }
+          });
+          if(li.hasClass("jstree-open")) { tmp1.state = "open"; }
+          if(li.hasClass("jstree-closed")) { tmp1.state = "closed"; }
+          if(li.data()) { tmp1.metadata = li.data(); }
+          a = li.children("a");
+          a.each(function () {
+            t = $(this);
+            if(
+              a_attr.length ||
+              $.inArray("languages", s.plugins) !== -1 ||
+              t.children("ins").get(0).style.backgroundImage.length ||
+              (t.children("ins").get(0).className && t.children("ins").get(0).className.replace(/jstree[^ ]*|$/ig,'').length)
+            ) {
+              lang = false;
+              if($.inArray("languages", s.plugins) !== -1 && $.isArray(s.languages) && s.languages.length) {
+                $.each(s.languages, function (l, lv) {
+                  if(t.hasClass(lv)) {
+                    lang = lv;
+                    return false;
+                  }
+                });
+              }
+              tmp2 = { attr : { }, title : _this.get_text(t, lang) };
+              $.each(a_attr, function (k, z) {
+                tmp2.attr[z] = (" " + (t.attr(z) || "")).replace(/ jstree[^ ]*/ig,'').replace(/\s+$/ig," ").replace(/^ /,"").replace(/ $/,"");
+              });
+              if($.inArray("languages", s.plugins) !== -1 && $.isArray(s.languages) && s.languages.length) {
+                $.each(s.languages, function (k, z) {
+                  if(t.hasClass(z)) { tmp2.language = z; return true; }
+                });
+              }
+              if(t.children("ins").get(0).className.replace(/jstree[^ ]*|$/ig,'').replace(/^\s+$/ig,"").length) {
+                tmp2.icon = t.children("ins").get(0).className.replace(/jstree[^ ]*|$/ig,'').replace(/\s+$/ig," ").replace(/^ /,"").replace(/ $/,"");
+              }
+              if(t.children("ins").get(0).style.backgroundImage.length) {
+                tmp2.icon = t.children("ins").get(0).style.backgroundImage.replace("url(","").replace(")","");
+              }
+            }
+            else {
+              tmp2 = _this.get_text(t);
+            }
+            if(a.length > 1) { tmp1.data.push(tmp2); }
+            else { tmp1.data = tmp2; }
+          });
+          li = li.find("> ul > li");
+          if(li.length) { tmp1.children = _this.get_json(li, li_attr, a_attr, true); }
+          result.push(tmp1);
+        });
+        return result;
+      }
+    }
+  });
+})(jQuery);
+//*/
+
+/*
+ * jsTree languages plugin
+ * Adds support for multiple language versions in one tree
+ * This basically allows for many titles coexisting in one node, but only one of them being visible at any given time
+ * This is useful for maintaining the same structure in many languages (hence the name of the plugin)
+ */
+(function ($) {
+  $.jstree.plugin("languages", {
+    __init : function () { this._load_css();  },
+    defaults : [],
+    _fn : {
+      set_lang : function (i) {
+        var langs = this._get_settings().languages,
+          st = false,
+          selector = ".jstree-" + this.get_index() + ' a';
+        if(!$.isArray(langs) || langs.length === 0) { return false; }
+        if($.inArray(i,langs) == -1) {
+          if(!!langs[i]) { i = langs[i]; }
+          else { return false; }
+        }
+        if(i == this.data.languages.current_language) { return true; }
+        st = $.vakata.css.get_css(selector + "." + this.data.languages.current_language, false, this.data.languages.language_css);
+        if(st !== false) { st.style.display = "none"; }
+        st = $.vakata.css.get_css(selector + "." + i, false, this.data.languages.language_css);
+        if(st !== false) { st.style.display = ""; }
+        this.data.languages.current_language = i;
+        this.__callback(i);
+        return true;
+      },
+      get_lang : function () {
+        return this.data.languages.current_language;
+      },
+      _get_string : function (key, lang) {
+        var langs = this._get_settings().languages,
+          s = this._get_settings().core.strings;
+        if($.isArray(langs) && langs.length) {
+          lang = (lang && $.inArray(lang,langs) != -1) ? lang : this.data.languages.current_language;
+        }
+        if(s[lang] && s[lang][key]) { return s[lang][key]; }
+        if(s[key]) { return s[key]; }
+        return key;
+      },
+      get_text : function (obj, lang) {
+        obj = this._get_node(obj) || this.data.ui.last_selected;
+        if(!obj.size()) { return false; }
+        var langs = this._get_settings().languages,
+          s = this._get_settings().core.html_titles;
+        if($.isArray(langs) && langs.length) {
+          lang = (lang && $.inArray(lang,langs) != -1) ? lang : this.data.languages.current_language;
+          obj = obj.children("a." + lang);
+        }
+        else { obj = obj.children("a:eq(0)"); }
+        if(s) {
+          obj = obj.clone();
+          obj.children("INS").remove();
+          return obj.html();
+        }
+        else {
+          obj = obj.contents().filter(function() { return this.nodeType == 3; })[0];
+          return obj.nodeValue;
+        }
+      },
+      set_text : function (obj, val, lang) {
+        obj = this._get_node(obj) || this.data.ui.last_selected;
+        if(!obj.size()) { return false; }
+        var langs = this._get_settings().languages,
+          s = this._get_settings().core.html_titles,
+          tmp;
+        if($.isArray(langs) && langs.length) {
+          lang = (lang && $.inArray(lang,langs) != -1) ? lang : this.data.languages.current_language;
+          obj = obj.children("a." + lang);
+        }
+        else { obj = obj.children("a:eq(0)"); }
+        if(s) {
+          tmp = obj.children("INS").clone();
+          obj.html(val).prepend(tmp);
+          this.__callback({ "obj" : obj, "name" : val, "lang" : lang });
+          return true;
+        }
+        else {
+          obj = obj.contents().filter(function() { return this.nodeType == 3; })[0];
+          this.__callback({ "obj" : obj, "name" : val, "lang" : lang });
+          return (obj.nodeValue = val);
+        }
+      },
+      _load_css : function () {
+        var langs = this._get_settings().languages,
+          str = "/* languages css */",
+          selector = ".jstree-" + this.get_index() + ' a',
+          ln;
+        if($.isArray(langs) && langs.length) {
+          this.data.languages.current_language = langs[0];
+          for(ln = 0; ln < langs.length; ln++) {
+            str += selector + "." + langs[ln] + " {";
+            if(langs[ln] != this.data.languages.current_language) { str += " display:none; "; }
+            str += " } ";
+          }
+          this.data.languages.language_css = $.vakata.css.add_sheet({ 'str' : str, 'title' : "jstree-languages" });
+        }
+      },
+      create_node : function (obj, position, js, callback) {
+        var t = this.__call_old(true, obj, position, js, function (t) {
+          var langs = this._get_settings().languages,
+            a = t.children("a"),
+            ln;
+          if($.isArray(langs) && langs.length) {
+            for(ln = 0; ln < langs.length; ln++) {
+              if(!a.is("." + langs[ln])) {
+                t.append(a.eq(0).clone().removeClass(langs.join(" ")).addClass(langs[ln]));
+              }
+            }
+            a.not("." + langs.join(", .")).remove();
+          }
+          if(callback) { callback.call(this, t); }
+        });
+        return t;
+      }
+    }
+  });
+})(jQuery);
+//*/
+
+/*
+ * jsTree cookies plugin
+ * Stores the currently opened/selected nodes in a cookie and then restores them
+ * Depends on the jquery.cookie plugin
+ */
+(function ($) {
+  $.jstree.plugin("cookies", {
+    __init : function () {
+      if(typeof $.cookie === "undefined") { throw "jsTree cookie: jQuery cookie plugin not included."; }
+
+      var s = this._get_settings().cookies,
+        tmp;
+      if(!!s.save_loaded) {
+        tmp = $.cookie(s.save_loaded);
+        if(tmp && tmp.length) { this.data.core.to_load = tmp.split(","); }
+      }
+      if(!!s.save_opened) {
+        tmp = $.cookie(s.save_opened);
+        if(tmp && tmp.length) { this.data.core.to_open = tmp.split(","); }
+      }
+      if(!!s.save_selected) {
+        tmp = $.cookie(s.save_selected);
+        if(tmp && tmp.length && this.data.ui) { this.data.ui.to_select = tmp.split(","); }
+      }
+      this.ge

<TRUNCATED>

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


[09/50] [abbrv] hadoop git commit: HDFS-11069. Tighten the authorization of datanode RPC. Contributed by Kihwal Lee

Posted by vi...@apache.org.
HDFS-11069. Tighten the authorization of datanode RPC. Contributed by Kihwal Lee


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

Branch: refs/heads/HDFS-9806
Commit: ae48c496dce8d0eae4571fc64e6850d602bae688
Parents: db41965
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Oct 27 14:17:16 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Oct 27 14:17:16 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae48c496/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 416c138..9ceffc2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -989,7 +989,7 @@ public class DataNode extends ReconfigurableBase
 
     // Is this by the DN user itself?
     assert dnUserName != null;
-    if (callerUgi.getShortUserName().equals(dnUserName)) {
+    if (callerUgi.getUserName().equals(dnUserName)) {
       return;
     }
 
@@ -1348,7 +1348,7 @@ public class DataNode extends ReconfigurableBase
     this.blockPoolTokenSecretManager = new BlockPoolTokenSecretManager();
 
     // Login is done by now. Set the DN user name.
-    dnUserName = UserGroupInformation.getCurrentUser().getShortUserName();
+    dnUserName = UserGroupInformation.getCurrentUser().getUserName();
     LOG.info("dnUserName = " + dnUserName);
     LOG.info("supergroup = " + supergroup);
     initIpcServer();


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


[22/50] [abbrv] hadoop git commit: YARN-4765 Split TestHBaseTimelineStorage into multiple test classes (Varun Saxena via Vrushali C)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/022bf783/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
deleted file mode 100644
index e37865f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ /dev/null
@@ -1,3751 +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.yarn.server.timelineservice.storage;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
-import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Various tests to test writing entities to HBase and reading them back from
- * it.
- *
- * It uses a single HBase mini-cluster for all tests which is a little more
- * realistic, and helps test correctness in the presence of other data.
- *
- * Each test uses a different cluster name to be able to handle its own data
- * even if other records exist in the table. Use a different cluster name if
- * you add a new test.
- */
-public class TestHBaseTimelineStorage {
-
-  private static HBaseTestingUtility util;
-  private HBaseTimelineReaderImpl reader;
-
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-    util = new HBaseTestingUtility();
-    util.startMiniCluster();
-    createSchema();
-    loadEntities();
-    loadApps();
-  }
-
-  private static void createSchema() throws IOException {
-    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
-  }
-
-  private static void loadApps() throws IOException {
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entity = new TimelineEntity();
-    String id = "application_1111111111_2222";
-    entity.setId(id);
-    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    Long cTime = 1425016502000L;
-    entity.setCreatedTime(cTime);
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue2");
-    infoMap.put("infoMapKey2", 20);
-    infoMap.put("infoMapKey3", 85.85);
-    entity.addInfo(infoMap);
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add("relatedto1");
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put("task", isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-    // add the relatesTo info
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add("relatesto1");
-    relatesToSet.add("relatesto3");
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put("container", relatesToSet);
-    Set<String> relatesToSet11 = new HashSet<String>();
-    relatesToSet11.add("relatesto4");
-    relatesTo.put("container1", relatesToSet11);
-    entity.setRelatesToEntities(relatesTo);
-    // add some config entries
-    Map<String, String> conf = new HashMap<String, String>();
-    conf.put("config_param1", "value1");
-    conf.put("config_param2", "value2");
-    conf.put("cfg_param1", "value3");
-    entity.addConfigs(conf);
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    TimelineMetric m1 = new TimelineMetric();
-    m1.setId("MAP_SLOT_MILLIS");
-    Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    long ts = System.currentTimeMillis();
-    metricValues.put(ts - 120000, 100000000);
-    metricValues.put(ts - 100000, 200000000);
-    metricValues.put(ts - 80000, 300000000);
-    metricValues.put(ts - 60000, 400000000);
-    metricValues.put(ts - 40000, 50000000000L);
-    metricValues.put(ts - 20000, 60000000000L);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-
-    TimelineMetric m12 = new TimelineMetric();
-    m12.setId("MAP1_BYTES");
-    m12.addValue(ts, 50);
-    metrics.add(m12);
-    entity.addMetrics(metrics);
-    TimelineEvent event = new TimelineEvent();
-    event.setId("start_event");
-    event.setTimestamp(ts);
-    entity.addEvent(event);
-    te.addEntity(entity);
-
-    TimelineEntities te1 = new TimelineEntities();
-    TimelineEntity entity1 = new TimelineEntity();
-    String id1 = "application_1111111111_3333";
-    entity1.setId(id1);
-    entity1.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    entity1.setCreatedTime(cTime + 20L);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap1 = new HashMap<String, Object>();
-    infoMap1.put("infoMapKey1", "infoMapValue1");
-    infoMap1.put("infoMapKey2", 10);
-    entity1.addInfo(infoMap1);
-
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet1 = new HashSet<String>();
-    isRelatedToSet1.add("relatedto3");
-    isRelatedToSet1.add("relatedto5");
-    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put("task1", isRelatedToSet1);
-    Set<String> isRelatedToSet11 = new HashSet<String>();
-    isRelatedToSet11.add("relatedto4");
-    isRelatedTo1.put("task2", isRelatedToSet11);
-    entity1.setIsRelatedToEntities(isRelatedTo1);
-
-    // add the relatesTo info
-    Set<String> relatesToSet1 = new HashSet<String>();
-    relatesToSet1.add("relatesto1");
-    relatesToSet1.add("relatesto2");
-    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put("container", relatesToSet1);
-    entity1.setRelatesToEntities(relatesTo1);
-
-    // add some config entries
-    Map<String, String> conf1 = new HashMap<String, String>();
-    conf1.put("cfg_param1", "value1");
-    conf1.put("cfg_param2", "value2");
-    entity1.addConfigs(conf1);
-
-    // add metrics
-    Set<TimelineMetric> metrics1 = new HashSet<>();
-    TimelineMetric m2 = new TimelineMetric();
-    m2.setId("MAP1_SLOT_MILLIS");
-    Map<Long, Number> metricValues1 = new HashMap<Long, Number>();
-    long ts1 = System.currentTimeMillis();
-    metricValues1.put(ts1 - 120000, 100000000);
-    metricValues1.put(ts1 - 100000, 200000000);
-    metricValues1.put(ts1 - 80000, 300000000);
-    metricValues1.put(ts1 - 60000, 400000000);
-    metricValues1.put(ts1 - 40000, 50000000000L);
-    metricValues1.put(ts1 - 20000, 60000000000L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues1);
-    metrics1.add(m2);
-    entity1.addMetrics(metrics1);
-    TimelineEvent event11 = new TimelineEvent();
-    event11.setId("end_event");
-    event11.setTimestamp(ts);
-    entity1.addEvent(event11);
-    TimelineEvent event12 = new TimelineEvent();
-    event12.setId("update_event");
-    event12.setTimestamp(ts - 10);
-    entity1.addEvent(event12);
-    te1.addEntity(entity1);
-
-    TimelineEntities te2 = new TimelineEntities();
-    TimelineEntity entity2 = new TimelineEntity();
-    String id2 = "application_1111111111_4444";
-    entity2.setId(id2);
-    entity2.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    entity2.setCreatedTime(cTime + 40L);
-    TimelineEvent event21 = new TimelineEvent();
-    event21.setId("update_event");
-    event21.setTimestamp(ts - 20);
-    entity2.addEvent(event21);
-    Set<String> isRelatedToSet2 = new HashSet<String>();
-    isRelatedToSet2.add("relatedto3");
-    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
-    isRelatedTo2.put("task1", isRelatedToSet2);
-    entity2.setIsRelatedToEntities(isRelatedTo2);
-    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
-    Set<String> relatesToSet14 = new HashSet<String>();
-    relatesToSet14.add("relatesto7");
-    relatesTo3.put("container2", relatesToSet14);
-    entity2.setRelatesToEntities(relatesTo3);
-
-    te2.addEntity(entity2);
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
-      hbi.init(util.getConfiguration());
-      hbi.start();
-      String cluster = "cluster1";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      String appName = "application_1111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-      appName = "application_1111111111_3333";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-      appName = "application_1111111111_4444";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te2);
-      hbi.stop();
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  private static void loadEntities() throws IOException {
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entity = new TimelineEntity();
-    String id = "hello";
-    String type = "world";
-    entity.setId(id);
-    entity.setType(type);
-    Long cTime = 1425016502000L;
-    entity.setCreatedTime(cTime);
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue2");
-    infoMap.put("infoMapKey2", 20);
-    infoMap.put("infoMapKey3", 71.4);
-    entity.addInfo(infoMap);
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add("relatedto1");
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put("task", isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-
-    // add the relatesTo info
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add("relatesto1");
-    relatesToSet.add("relatesto3");
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put("container", relatesToSet);
-    Set<String> relatesToSet11 = new HashSet<String>();
-    relatesToSet11.add("relatesto4");
-    relatesTo.put("container1", relatesToSet11);
-    entity.setRelatesToEntities(relatesTo);
-
-    // add some config entries
-    Map<String, String> conf = new HashMap<String, String>();
-    conf.put("config_param1", "value1");
-    conf.put("config_param2", "value2");
-    conf.put("cfg_param1", "value3");
-    entity.addConfigs(conf);
-
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    TimelineMetric m1 = new TimelineMetric();
-    m1.setId("MAP_SLOT_MILLIS");
-    Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    long ts = System.currentTimeMillis();
-    metricValues.put(ts - 120000, 100000000);
-    metricValues.put(ts - 100000, 200000000);
-    metricValues.put(ts - 80000, 300000000);
-    metricValues.put(ts - 60000, 400000000);
-    metricValues.put(ts - 40000, 50000000000L);
-    metricValues.put(ts - 20000, 70000000000L);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-
-    TimelineMetric m12 = new TimelineMetric();
-    m12.setId("MAP1_BYTES");
-    m12.addValue(ts, 50);
-    metrics.add(m12);
-    entity.addMetrics(metrics);
-    TimelineEvent event = new TimelineEvent();
-    event.setId("start_event");
-    event.setTimestamp(ts);
-    entity.addEvent(event);
-    te.addEntity(entity);
-
-    TimelineEntity entity1 = new TimelineEntity();
-    String id1 = "hello1";
-    entity1.setId(id1);
-    entity1.setType(type);
-    entity1.setCreatedTime(cTime + 20L);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap1 = new HashMap<String, Object>();
-    infoMap1.put("infoMapKey1", "infoMapValue1");
-    infoMap1.put("infoMapKey2", 10);
-    entity1.addInfo(infoMap1);
-
-    // add event.
-    TimelineEvent event11 = new TimelineEvent();
-    event11.setId("end_event");
-    event11.setTimestamp(ts);
-    entity1.addEvent(event11);
-    TimelineEvent event12 = new TimelineEvent();
-    event12.setId("update_event");
-    event12.setTimestamp(ts - 10);
-    entity1.addEvent(event12);
-
-
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet1 = new HashSet<String>();
-    isRelatedToSet1.add("relatedto3");
-    isRelatedToSet1.add("relatedto5");
-    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put("task1", isRelatedToSet1);
-    Set<String> isRelatedToSet11 = new HashSet<String>();
-    isRelatedToSet11.add("relatedto4");
-    isRelatedTo1.put("task2", isRelatedToSet11);
-    entity1.setIsRelatedToEntities(isRelatedTo1);
-
-    // add the relatesTo info
-    Set<String> relatesToSet1 = new HashSet<String>();
-    relatesToSet1.add("relatesto1");
-    relatesToSet1.add("relatesto2");
-    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put("container", relatesToSet1);
-    entity1.setRelatesToEntities(relatesTo1);
-
-    // add some config entries
-    Map<String, String> conf1 = new HashMap<String, String>();
-    conf1.put("cfg_param1", "value1");
-    conf1.put("cfg_param2", "value2");
-    entity1.addConfigs(conf1);
-
-    // add metrics
-    Set<TimelineMetric> metrics1 = new HashSet<>();
-    TimelineMetric m2 = new TimelineMetric();
-    m2.setId("MAP1_SLOT_MILLIS");
-    Map<Long, Number> metricValues1 = new HashMap<Long, Number>();
-    long ts1 = System.currentTimeMillis();
-    metricValues1.put(ts1 - 120000, 100000000);
-    metricValues1.put(ts1 - 100000, 200000000);
-    metricValues1.put(ts1 - 80000, 300000000);
-    metricValues1.put(ts1 - 60000, 400000000);
-    metricValues1.put(ts1 - 40000, 50000000000L);
-    metricValues1.put(ts1 - 20000, 60000000000L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues1);
-    metrics1.add(m2);
-    entity1.addMetrics(metrics1);
-    te.addEntity(entity1);
-
-    TimelineEntity entity2 = new TimelineEntity();
-    String id2 = "hello2";
-    entity2.setId(id2);
-    entity2.setType(type);
-    entity2.setCreatedTime(cTime + 40L);
-    TimelineEvent event21 = new TimelineEvent();
-    event21.setId("update_event");
-    event21.setTimestamp(ts - 20);
-    entity2.addEvent(event21);
-    Set<String> isRelatedToSet2 = new HashSet<String>();
-    isRelatedToSet2.add("relatedto3");
-    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
-    isRelatedTo2.put("task1", isRelatedToSet2);
-    entity2.setIsRelatedToEntities(isRelatedTo2);
-    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
-    Set<String> relatesToSet14 = new HashSet<String>();
-    relatesToSet14.add("relatesto7");
-    relatesTo3.put("container2", relatesToSet14);
-    entity2.setRelatesToEntities(relatesTo3);
-    te.addEntity(entity2);
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
-      hbi.init(util.getConfiguration());
-      hbi.start();
-      String cluster = "cluster1";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      String appName = "application_1231111111_1111";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-      hbi.stop();
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Before
-  public void init() throws Exception {
-    reader = new HBaseTimelineReaderImpl();
-    reader.init(util.getConfiguration());
-    reader.start();
-  }
-
-  @After
-  public void stop() throws Exception {
-    if (reader != null) {
-      reader.stop();
-      reader.close();
-    }
-  }
-
-  private static void matchMetrics(Map<Long, Number> m1, Map<Long, Number> m2) {
-    assertEquals(m1.size(), m2.size());
-    for (Map.Entry<Long, Number> entry : m2.entrySet()) {
-      Number val = m1.get(entry.getKey());
-      assertNotNull(val);
-      assertEquals(val.longValue(), entry.getValue().longValue());
-    }
-  }
-
-  @Test
-  public void testWriteNullApplicationToHBase() throws Exception {
-    TimelineEntities te = new TimelineEntities();
-    ApplicationEntity entity = new ApplicationEntity();
-    String appId = "application_1000178881110_2002";
-    entity.setId(appId);
-    long cTime = 1425016501000L;
-    entity.setCreatedTime(cTime);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("in fo M apK  ey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
-    entity.addInfo(infoMap);
-
-    te.addEntity(entity);
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_check_null_application";
-      String user = "user1check_null_application";
-      //set the flow name to null
-      String flow = null;
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
-      hbi.stop();
-
-      // retrieve the row
-      Scan scan = new Scan();
-      scan.setStartRow(Bytes.toBytes(cluster));
-      scan.setStopRow(Bytes.toBytes(cluster + "1"));
-      Connection conn = ConnectionFactory.createConnection(c1);
-      ResultScanner resultScanner = new ApplicationTable()
-          .getResultScanner(c1, conn, scan);
-
-      assertTrue(resultScanner != null);
-      // try to iterate over results
-      int count = 0;
-      for (Result rr = resultScanner.next(); rr != null;
-          rr = resultScanner.next()) {
-        count++;
-      }
-      // there should be no rows written
-      // no exceptions thrown during write
-      assertEquals(0, count);
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Test
-  public void testWriteApplicationToHBase() throws Exception {
-    TimelineEntities te = new TimelineEntities();
-    ApplicationEntity entity = new ApplicationEntity();
-    String appId = "application_1000178881110_2002";
-    entity.setId(appId);
-    Long cTime = 1425016501000L;
-    entity.setCreatedTime(cTime);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
-    entity.addInfo(infoMap);
-
-    // add the isRelatedToEntity info
-    String key = "task";
-    String value = "is_related_to_entity_id_here";
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add(value);
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put(key, isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-
-    // add the relatesTo info
-    key = "container";
-    value = "relates_to_entity_id_here";
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add(value);
-    value = "relates_to_entity_id_here_Second";
-    relatesToSet.add(value);
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put(key, relatesToSet);
-    entity.setRelatesToEntities(relatesTo);
-
-    // add some config entries
-    Map<String, String> conf = new HashMap<String, String>();
-    conf.put("config_param1", "value1");
-    conf.put("config_param2", "value2");
-    entity.addConfigs(conf);
-
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    TimelineMetric m1 = new TimelineMetric();
-    m1.setId("MAP_SLOT_MILLIS");
-    Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    long ts = System.currentTimeMillis();
-    metricValues.put(ts - 120000, 100000000);
-    metricValues.put(ts - 100000, 200000000);
-    metricValues.put(ts - 80000, 300000000);
-    metricValues.put(ts - 60000, 400000000);
-    metricValues.put(ts - 40000, 50000000000L);
-    metricValues.put(ts - 20000, 60000000000L);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity.addMetrics(metrics);
-
-    // add aggregated metrics
-    TimelineEntity aggEntity = new TimelineEntity();
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    aggEntity.setId(appId);
-    aggEntity.setType(type);
-    long cTime2 = 1425016502000L;
-    aggEntity.setCreatedTime(cTime2);
-
-    TimelineMetric aggMetric = new TimelineMetric();
-    aggMetric.setId("MEM_USAGE");
-    Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
-    long aggTs = ts;
-    aggMetricValues.put(aggTs - 120000, 102400000L);
-    aggMetric.setType(Type.SINGLE_VALUE);
-    aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
-    aggMetric.setValues(aggMetricValues);
-    Set<TimelineMetric> aggMetrics = new HashSet<>();
-    aggMetrics.add(aggMetric);
-    entity.addMetrics(aggMetrics);
-    te.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_write_app";
-      String user = "user1";
-      String flow = "s!ome_f\tlow  _n am!e";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
-
-      // Write entity again, this time without created time.
-      entity = new ApplicationEntity();
-      appId = "application_1000178881110_2002";
-      entity.setId(appId);
-      // add the info map in Timeline Entity
-      Map<String, Object> infoMap1 = new HashMap<>();
-      infoMap1.put("infoMapKey3", "infoMapValue1");
-      entity.addInfo(infoMap1);
-      te = new TimelineEntities();
-      te.addEntity(entity);
-      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
-      hbi.stop();
-
-      infoMap.putAll(infoMap1);
-      // retrieve the row
-      ApplicationRowKey applicationRowKey =
-          new ApplicationRowKey(cluster, user, flow, runid, appId);
-      byte[] rowKey = applicationRowKey.getRowKey();
-      Get get = new Get(rowKey);
-      get.setMaxVersions(Integer.MAX_VALUE);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      Result result = new ApplicationTable().getResult(c1, conn, get);
-
-      assertTrue(result != null);
-      assertEquals(17, result.size());
-
-      // check the row key
-      byte[] row1 = result.getRow();
-      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
-          appId));
-
-      // check info column family
-      String id1 = ApplicationColumn.ID.readResult(result).toString();
-      assertEquals(appId, id1);
-
-      Long cTime1 =
-          (Long) ApplicationColumn.CREATED_TIME.readResult(result);
-      assertEquals(cTime, cTime1);
-
-      Map<String, Object> infoColumns =
-          ApplicationColumnPrefix.INFO.readResults(result,
-              new StringKeyConverter());
-      assertEquals(infoMap, infoColumns);
-
-      // Remember isRelatedTo is of type Map<String, Set<String>>
-      for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
-          .entrySet()) {
-        Object isRelatedToValue =
-            ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
-                isRelatedToEntry.getKey());
-        String compoundValue = isRelatedToValue.toString();
-        // id7?id9?id6
-        Set<String> isRelatedToValues =
-            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
-        assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
-            isRelatedToValues.size());
-        for (String v : isRelatedToEntry.getValue()) {
-          assertTrue(isRelatedToValues.contains(v));
-        }
-      }
-
-      // RelatesTo
-      for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
-          .entrySet()) {
-        String compoundValue =
-            ApplicationColumnPrefix.RELATES_TO.readResult(result,
-                relatesToEntry.getKey()).toString();
-        // id3?id4?id5
-        Set<String> relatesToValues =
-            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
-        assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
-            relatesToValues.size());
-        for (String v : relatesToEntry.getValue()) {
-          assertTrue(relatesToValues.contains(v));
-        }
-      }
-
-      KeyConverter<String> stringKeyConverter = new StringKeyConverter();
-      // Configuration
-      Map<String, Object> configColumns =
-          ApplicationColumnPrefix.CONFIG
-              .readResults(result, stringKeyConverter);
-      assertEquals(conf, configColumns);
-
-      NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result,
-              stringKeyConverter);
-
-      NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-      matchMetrics(metricValues, metricMap);
-
-      // read the timeline entity using the reader this time. In metrics limit
-      // specify Integer MAX_VALUE. A TIME_SERIES will be returned(if more than
-      // one value exists for a metric).
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appId,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null,
-          EnumSet.of(TimelineReader.Field.ALL), Integer.MAX_VALUE));
-      assertNotNull(e1);
-
-      // verify attributes
-      assertEquals(appId, e1.getId());
-      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
-          e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      Map<String, Object> infoMap2 = e1.getInfo();
-      assertEquals(infoMap, infoMap2);
-
-      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
-      assertEquals(isRelatedTo, isRelatedTo2);
-
-      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
-      assertEquals(relatesTo, relatesTo2);
-
-      Map<String, String> conf2 = e1.getConfigs();
-      assertEquals(conf, conf2);
-
-      Set<TimelineMetric> metrics2 = e1.getMetrics();
-      assertEquals(2, metrics2.size());
-      for (TimelineMetric metric2 : metrics2) {
-        Map<Long, Number> metricValues2 = metric2.getValues();
-        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
-            metric2.getId().equals("MEM_USAGE"));
-        if (metric2.getId().equals("MAP_SLOT_MILLIS")) {
-          assertEquals(6, metricValues2.size());
-          matchMetrics(metricValues, metricValues2);
-        }
-        if (metric2.getId().equals("MEM_USAGE")) {
-          assertEquals(1, metricValues2.size());
-          matchMetrics(aggMetricValues, metricValues2);
-        }
-      }
-
-      // In metrics limit specify a value of 3. No more than 3 values for a
-      // metric will be returned.
-      e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
-          runid, appId, entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null,
-          EnumSet.of(TimelineReader.Field.ALL), 3));
-      assertNotNull(e1);
-      assertEquals(appId, e1.getId());
-      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
-          e1.getType());
-      assertEquals(conf, e1.getConfigs());
-      metrics2 = e1.getMetrics();
-      assertEquals(2, metrics2.size());
-      for (TimelineMetric metric2 : metrics2) {
-        Map<Long, Number> metricValues2 = metric2.getValues();
-        assertTrue(metricValues2.size() <= 3);
-        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
-            metric2.getId().equals("MEM_USAGE"));
-      }
-
-      // Check if single value(latest value) instead of time series is returned
-      // if metricslimit is not set(null), irrespective of number of metric
-      // values.
-      e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appId,
-         entity.getType(), entity.getId()), new TimelineDataToRetrieve(
-         null, null, EnumSet.of(TimelineReader.Field.ALL), null));
-      assertNotNull(e1);
-      assertEquals(appId, e1.getId());
-      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
-          e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      assertEquals(infoMap, e1.getInfo());
-      assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
-      assertEquals(relatesTo, e1.getRelatesToEntities());
-      assertEquals(conf, e1.getConfigs());
-      assertEquals(2, e1.getMetrics().size());
-      for (TimelineMetric metric : e1.getMetrics()) {
-        assertEquals(1, metric.getValues().size());
-        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
-        assertTrue(metric.getId().equals("MAP_SLOT_MILLIS") ||
-            metric.getId().equals("MEM_USAGE"));
-        assertEquals(1, metric.getValues().size());
-        if (metric.getId().equals("MAP_SLOT_MILLIS")) {
-          assertTrue(metric.getValues().containsKey(ts - 20000));
-          assertEquals(metricValues.get(ts - 20000),
-              metric.getValues().get(ts - 20000));
-        }
-        if (metric.getId().equals("MEM_USAGE")) {
-          assertTrue(metric.getValues().containsKey(aggTs - 120000));
-          assertEquals(aggMetricValues.get(aggTs - 120000),
-              metric.getValues().get(aggTs - 120000));
-        }
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Test
-  public void testWriteEntityToHBase() throws Exception {
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entity = new TimelineEntity();
-    String id = "hello";
-    String type = "world";
-    entity.setId(id);
-    entity.setType(type);
-    Long cTime = 1425016501000L;
-    entity.setCreatedTime(cTime);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
-    entity.addInfo(infoMap);
-
-    // add the isRelatedToEntity info
-    String key = "task";
-    String value = "is_related_to_entity_id_here";
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add(value);
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put(key, isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-
-    // add the relatesTo info
-    key = "container";
-    value = "relates_to_entity_id_here";
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add(value);
-    value = "relates_to_entity_id_here_Second";
-    relatesToSet.add(value);
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put(key, relatesToSet);
-    entity.setRelatesToEntities(relatesTo);
-
-    // add some config entries
-    Map<String, String> conf = new HashMap<String, String>();
-    conf.put("config_param1", "value1");
-    conf.put("config_param2", "value2");
-    entity.addConfigs(conf);
-
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    TimelineMetric m1 = new TimelineMetric();
-    m1.setId("MAP_SLOT_MILLIS");
-    Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    long ts = System.currentTimeMillis();
-    metricValues.put(ts - 120000, 100000000);
-    metricValues.put(ts - 100000, 200000000);
-    metricValues.put(ts - 80000, 300000000);
-    metricValues.put(ts - 60000, 400000000);
-    metricValues.put(ts - 40000, 50000000000L);
-    metricValues.put(ts - 20000, 60000000000L);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity.addMetrics(metrics);
-    te.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_write_entity";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-      hbi.stop();
-
-      // scan the table and see that entity exists
-      Scan s = new Scan();
-      byte[] startRow =
-          new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
-              .getRowKeyPrefix();
-      s.setStartRow(startRow);
-      s.setMaxVersions(Integer.MAX_VALUE);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
-
-      int rowCount = 0;
-      int colCount = 0;
-      KeyConverter<String> stringKeyConverter = new StringKeyConverter();
-      for (Result result : scanner) {
-        if (result != null && !result.isEmpty()) {
-          rowCount++;
-          colCount += result.size();
-          byte[] row1 = result.getRow();
-          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
-              entity));
-
-          // check info column family
-          String id1 = EntityColumn.ID.readResult(result).toString();
-          assertEquals(id, id1);
-
-          String type1 = EntityColumn.TYPE.readResult(result).toString();
-          assertEquals(type, type1);
-
-          Long cTime1 = (Long) EntityColumn.CREATED_TIME.readResult(result);
-          assertEquals(cTime1, cTime);
-
-          Map<String, Object> infoColumns =
-              EntityColumnPrefix.INFO.readResults(result,
-                  new StringKeyConverter());
-          assertEquals(infoMap, infoColumns);
-
-          // Remember isRelatedTo is of type Map<String, Set<String>>
-          for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
-              .entrySet()) {
-            Object isRelatedToValue =
-                EntityColumnPrefix.IS_RELATED_TO.readResult(result,
-                    isRelatedToEntry.getKey());
-            String compoundValue = isRelatedToValue.toString();
-            // id7?id9?id6
-            Set<String> isRelatedToValues =
-                new HashSet<String>(
-                    Separator.VALUES.splitEncoded(compoundValue));
-            assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
-                isRelatedToValues.size());
-            for (String v : isRelatedToEntry.getValue()) {
-              assertTrue(isRelatedToValues.contains(v));
-            }
-          }
-
-          // RelatesTo
-          for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
-              .entrySet()) {
-            String compoundValue = EntityColumnPrefix.RELATES_TO
-                .readResult(result, relatesToEntry.getKey()).toString();
-            // id3?id4?id5
-            Set<String> relatesToValues =
-                new HashSet<String>(
-                    Separator.VALUES.splitEncoded(compoundValue));
-            assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
-                relatesToValues.size());
-            for (String v : relatesToEntry.getValue()) {
-              assertTrue(relatesToValues.contains(v));
-            }
-          }
-
-          // Configuration
-          Map<String, Object> configColumns =
-              EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter);
-          assertEquals(conf, configColumns);
-
-          NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result,
-                  stringKeyConverter);
-
-          NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-          matchMetrics(metricValues, metricMap);
-        }
-      }
-      assertEquals(1, rowCount);
-      assertEquals(16, colCount);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
-          Integer.MAX_VALUE));
-      Set<TimelineEntity> es1 = reader.getEntities(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
-          Integer.MAX_VALUE));
-      assertNotNull(e1);
-      assertEquals(1, es1.size());
-
-      // verify attributes
-      assertEquals(id, e1.getId());
-      assertEquals(type, e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      Map<String, Object> infoMap2 = e1.getInfo();
-      assertEquals(infoMap, infoMap2);
-
-      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
-      assertEquals(isRelatedTo, isRelatedTo2);
-
-      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
-      assertEquals(relatesTo, relatesTo2);
-
-      Map<String, String> conf2 = e1.getConfigs();
-      assertEquals(conf, conf2);
-
-      Set<TimelineMetric> metrics2 = e1.getMetrics();
-      assertEquals(metrics, metrics2);
-      for (TimelineMetric metric2 : metrics2) {
-        Map<Long, Number> metricValues2 = metric2.getValues();
-        matchMetrics(metricValues, metricValues2);
-      }
-
-      e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
-          runid, appName, entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-      assertNotNull(e1);
-      assertEquals(id, e1.getId());
-      assertEquals(type, e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      assertEquals(infoMap, e1.getInfo());
-      assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
-      assertEquals(relatesTo, e1.getRelatesToEntities());
-      assertEquals(conf, e1.getConfigs());
-      for (TimelineMetric metric : e1.getMetrics()) {
-        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
-        assertEquals(1, metric.getValues().size());
-        assertTrue(metric.getValues().containsKey(ts - 20000));
-        assertEquals(metricValues.get(ts - 20000),
-            metric.getValues().get(ts - 20000));
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
-      String flow, Long runid, String appName, TimelineEntity te) {
-
-    EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
-
-    assertEquals(user, key.getUserId());
-    assertEquals(cluster, key.getClusterId());
-    assertEquals(flow, key.getFlowName());
-    assertEquals(runid, key.getFlowRunId());
-    assertEquals(appName, key.getAppId());
-    assertEquals(te.getType(), key.getEntityType());
-    assertEquals(te.getId(), key.getEntityId());
-    return true;
-  }
-
-  private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
-      String user, String flow, Long runid, String appName) {
-
-    ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
-
-    assertEquals(cluster, key.getClusterId());
-    assertEquals(user, key.getUserId());
-    assertEquals(flow, key.getFlowName());
-    assertEquals(runid, key.getFlowRunId());
-    assertEquals(appName, key.getAppId());
-    return true;
-  }
-
-  @Test
-  public void testEvents() throws IOException {
-    TimelineEvent event = new TimelineEvent();
-    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
-    event.setId(eventId);
-    Long expTs = 1436512802000L;
-    event.setTimestamp(expTs);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-
-    final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
-    entity.addEvent(event);
-
-    TimelineEntities entities = new TimelineEntities();
-    entities.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_events";
-      String user = "user2";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      String appName = "application_123465899910_1001";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
-      hbi.stop();
-
-      // retrieve the row
-      ApplicationRowKey applicationRowKey =
-          new ApplicationRowKey(cluster, user, flow, runid, appName);
-      byte[] rowKey = applicationRowKey.getRowKey();
-      Get get = new Get(rowKey);
-      get.setMaxVersions(Integer.MAX_VALUE);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      Result result = new ApplicationTable().getResult(c1, conn, get);
-
-      assertTrue(result != null);
-
-      // check the row key
-      byte[] row1 = result.getRow();
-      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
-          appName));
-
-      Map<EventColumnName, Object> eventsResult =
-          ApplicationColumnPrefix.EVENT.readResults(result,
-              new EventColumnNameConverter());
-      // there should be only one event
-      assertEquals(1, eventsResult.size());
-      for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
-        EventColumnName eventColumnName = e.getKey();
-        // the qualifier is a compound key
-        // hence match individual values
-        assertEquals(eventId, eventColumnName.getId());
-        assertEquals(expTs, eventColumnName.getTimestamp());
-        assertEquals(expKey, eventColumnName.getInfoKey());
-        Object value = e.getValue();
-        // there should be only one timestamp and value
-        assertEquals(expVal, value.toString());
-      }
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-      TimelineEntity e2 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, null, null, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-      assertNotNull(e1);
-      assertNotNull(e2);
-      assertEquals(e1, e2);
-
-      // check the events
-      NavigableSet<TimelineEvent> events = e1.getEvents();
-      // there should be only one event
-      assertEquals(1, events.size());
-      for (TimelineEvent e : events) {
-        assertEquals(eventId, e.getId());
-        assertEquals(expTs, Long.valueOf(e.getTimestamp()));
-        Map<String, Object> info = e.getInfo();
-        assertEquals(1, info.size());
-        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
-          assertEquals(expKey, infoEntry.getKey());
-          assertEquals(expVal, infoEntry.getValue());
-        }
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Test
-  public void testEventsWithEmptyInfo() throws IOException {
-    TimelineEvent event = new TimelineEvent();
-    String eventId = "foo_ev e  nt_id";
-    event.setId(eventId);
-    Long expTs = 1436512802000L;
-    event.setTimestamp(expTs);
-
-    final TimelineEntity entity = new TimelineEntity();
-    entity.setId("attempt_1329348432655_0001_m_000008_18");
-    entity.setType("FOO_ATTEMPT");
-    entity.addEvent(event);
-
-    TimelineEntities entities = new TimelineEntities();
-    entities.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_empty_eventkey";
-      String user = "user_emptyeventkey";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
-      byte[] startRow =
-          new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
-              .getRowKeyPrefix();
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
-      hbi.stop();
-      // scan the table and see that entity exists
-      Scan s = new Scan();
-      s.setStartRow(startRow);
-      s.addFamily(EntityColumnFamily.INFO.getBytes());
-      Connection conn = ConnectionFactory.createConnection(c1);
-      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
-
-      int rowCount = 0;
-      for (Result result : scanner) {
-        if (result != null && !result.isEmpty()) {
-          rowCount++;
-
-          // check the row key
-          byte[] row1 = result.getRow();
-          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
-              entity));
-
-          Map<EventColumnName, Object> eventsResult =
-              EntityColumnPrefix.EVENT.readResults(result,
-                  new EventColumnNameConverter());
-          // there should be only one event
-          assertEquals(1, eventsResult.size());
-          for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
-            EventColumnName eventColumnName = e.getKey();
-            // the qualifier is a compound key
-            // hence match individual values
-            assertEquals(eventId, eventColumnName.getId());
-            assertEquals(expTs, eventColumnName.getTimestamp());
-            // key must be empty
-            assertNull(eventColumnName.getInfoKey());
-            Object value = e.getValue();
-            // value should be empty
-            assertEquals("", value.toString());
-          }
-        }
-      }
-      assertEquals(1, rowCount);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-      Set<TimelineEntity> es1 = reader.getEntities(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-      assertNotNull(e1);
-      assertEquals(1, es1.size());
-
-      // check the events
-      NavigableSet<TimelineEvent> events = e1.getEvents();
-      // there should be only one event
-      assertEquals(1, events.size());
-      for (TimelineEvent e : events) {
-        assertEquals(eventId, e.getId());
-        assertEquals(expTs, Long.valueOf(e.getTimestamp()));
-        Map<String, Object> info = e.getInfo();
-        assertTrue(info == null || info.isEmpty());
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Test
-  public void testEventsEscapeTs() throws IOException {
-    TimelineEvent event = new TimelineEvent();
-    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
-    event.setId(eventId);
-    long expTs = 1463567041056L;
-    event.setTimestamp(expTs);
-    String expKey = "f==o o_e ve\tnt";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-
-    final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
-    entity.addEvent(event);
-
-    TimelineEntities entities = new TimelineEntities();
-    entities.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "clus!ter_\ttest_ev  ents";
-      String user = "user2";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      String appName = "application_123465899910_2001";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
-      hbi.stop();
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-      assertNotNull(e1);
-      // check the events
-      NavigableSet<TimelineEvent> events = e1.getEvents();
-      // there should be only one event
-      assertEquals(1, events.size());
-      for (TimelineEvent e : events) {
-        assertEquals(eventId, e.getId());
-        assertEquals(expTs, e.getTimestamp());
-        Map<String, Object> info = e.getInfo();
-        assertEquals(1, info.size());
-        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
-          assertEquals(expKey, infoEntry.getKey());
-          assertEquals(expVal, infoEntry.getValue());
-        }
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Test
-  public void testNonIntegralMetricValues() throws IOException {
-    TimelineEntities teApp = new TimelineEntities();
-    ApplicationEntity entityApp = new ApplicationEntity();
-    String appId = "application_1000178881110_2002";
-    entityApp.setId(appId);
-    entityApp.setCreatedTime(1425016501000L);
-    // add metrics with floating point values
-    Set<TimelineMetric> metricsApp = new HashSet<>();
-    TimelineMetric mApp = new TimelineMetric();
-    mApp.setId("MAP_SLOT_MILLIS");
-    Map<Long, Number> metricAppValues = new HashMap<Long, Number>();
-    long ts = System.currentTimeMillis();
-    metricAppValues.put(ts - 20, 10.5);
-    metricAppValues.put(ts - 10, 20.5);
-    mApp.setType(Type.TIME_SERIES);
-    mApp.setValues(metricAppValues);
-    metricsApp.add(mApp);
-    entityApp.addMetrics(metricsApp);
-    teApp.addEntity(entityApp);
-
-    TimelineEntities teEntity = new TimelineEntities();
-    TimelineEntity entity = new TimelineEntity();
-    entity.setId("hello");
-    entity.setType("world");
-    entity.setCreatedTime(1425016501000L);
-    // add metrics with floating point values
-    Set<TimelineMetric> metricsEntity = new HashSet<>();
-    TimelineMetric mEntity = new TimelineMetric();
-    mEntity.setId("MAP_SLOT_MILLIS");
-    mEntity.addValue(ts - 20, 10.5);
-    metricsEntity.add(mEntity);
-    entity.addMetrics(metricsEntity);
-    teEntity.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      // Writing application entity.
-      try {
-        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teApp);
-        Assert.fail("Expected an exception as metric values are non integral");
-      } catch (IOException e) {}
-
-      // Writing generic entity.
-      try {
-        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teEntity);
-        Assert.fail("Expected an exception as metric values are non integral");
-      } catch (IOException e) {}
-      hbi.stop();
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  @Test
-  public void testReadEntities() throws Exception {
-    TimelineEntity entity = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-    assertNotNull(entity);
-    assertEquals(3, entity.getConfigs().size());
-    assertEquals(1, entity.getIsRelatedToEntities().size());
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world",
-        null), new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-    assertEquals(3, entities.size());
-    int cfgCnt = 0;
-    int metricCnt = 0;
-    int infoCnt = 0;
-    int eventCnt = 0;
-    int relatesToCnt = 0;
-    int isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
-          timelineEntity.getConfigs().size();
-      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
-          timelineEntity.getMetrics().size();
-      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
-          timelineEntity.getInfo().size();
-      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
-          timelineEntity.getEvents().size();
-      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
-          timelineEntity.getRelatesToEntities().size();
-      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
-          timelineEntity.getIsRelatedToEntities().size();
-    }
-    assertEquals(5, cfgCnt);
-    assertEquals(3, metricCnt);
-    assertEquals(5, infoCnt);
-    assertEquals(4, eventCnt);
-    assertEquals(4, relatesToCnt);
-    assertEquals(4, isRelatedToCnt);
-  }
-
-  @Test
-  public void testFilterEntitiesByCreatedTime() throws Exception {
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
-        null, null, null, null, null), new TimelineDataToRetrieve());
-    assertEquals(3, entities.size());
-    for (TimelineEntity entity : entities) {
-      if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
-          !entity.getId().equals("hello2")) {
-        Assert.fail("Entities with ids' hello, hello1 and hello2 should be" +
-            " present");
-      }
-    }
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
-        null, null, null), new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    for (TimelineEntity entity : entities) {
-      if (!entity.getId().equals("hello1") &&
-          !entity.getId().equals("hello2")) {
-        Assert.fail("Entities with ids' hello1 and hello2 should be present");
-      }
-    }
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world",  null),
-        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
-        null, null, null), new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    for (TimelineEntity entity : entities) {
-      if (!entity.getId().equals("hello")) {
-        Assert.fail("Entity with id hello should be present");
-      }
-    }
-  }
-
-  @Test
-  public void testReadEntitiesRelationsAndEventFiltersDefaultView()
-      throws Exception {
-    TimelineFilterList eventFilter = new TimelineFilterList();
-    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
-        "end_event"));
-    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
-    relatesTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    relatesTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList isRelatedTo = new TimelineFilterList();
-    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3"))));
-    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto5"))));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
-        null, null, null, eventFilter), new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    int eventCnt = 0;
-    int isRelatedToCnt = 0;
-    int relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(0, eventCnt);
-    assertEquals(0, isRelatedToCnt);
-    assertEquals(0, relatesToCnt);
-  }
-
-  @Test
-  public void testReadEntitiesEventFilters() throws Exception {
-    TimelineFilterList ef = new TimelineFilterList();
-    ef.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    ef.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-    assertEquals(1, entities.size());
-    int eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(1, eventCnt);
-
-    TimelineFilterList ef1 = new TimelineFilterList();
-    ef1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    ef1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef1),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(0, eventCnt);
-
-    TimelineFilterList ef2 = new TimelineFilterList();
-    ef2.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef2),
-        new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity ids' should have been hello and hello2");
-      }
-    }
-    assertEquals(0, eventCnt);
-
-    TimelineFilterList ef3 = new TimelineFilterList();
-    ef3.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    ef3.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "dummy_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef3),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    list1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "dummy_event"));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "start_event"));
-    TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef4),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, eventCnt);
-
-    TimelineFilterList ef5 = new TimelineFilterList();
-    ef5.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "update_event"));
-    ef5.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef5),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, eventCnt);
-  }
-
-  @Test
-  public void testReadEntitiesIsRelatedTo() throws Exception {
-    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
-    irt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task",
-        new HashSet<Object>(Arrays.asList("relatedto1"))));
-    irt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task2",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
-        null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-    assertEquals(2, entities.size());
-    int isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity ids' should have been hello and hello1");
-      }
-    }
-    assertEquals(3, isRelatedToCnt);
-
-    TimelineFilterList irt1 = new TimelineFilterList();
-    irt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3"))));
-    irt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-
-    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
-    irt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task",
-        new HashSet<Object>(Arrays.asList("relatedto1"))));
-    irt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task2",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity ids' should have been hello and hello1");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-
-    TimelineFilterList irt3 = new TimelineFilterList();
-    irt3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-
-    TimelineFilterList irt4 = new TimelineFilterList();
-    irt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3"))));
-    irt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_task",
-        new HashSet<Object>(Arrays.asList("relatedto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList irt5 = new TimelineFilterList();
-    irt5.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task",
-        new HashSet<Object>(Arrays.asList("relatedto1"))));
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_task",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task2",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-  }
-
-  @Test
-  public void testReadEntitiesRelatesTo() throws Exception {
-    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
-    rt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    rt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
-    assertEquals(2, entities.size());
-    int relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity ids' should have been hello and hello2");
-      }
-    }
-    assertEquals(3, relatesToCnt);
-
-    TimelineFilterList rt1 = new TimelineFilterList();
-    rt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    rt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto3"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
-    rt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    rt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity ids' should have been hello and hello2");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList rt3 = new TimelineFilterList();
-    rt3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList rt4 = new TimelineFilterList();
-    rt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    rt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_container",
-        new HashSet<Object>(Arrays.asList("relatesto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList rt5 = new TimelineFilterList();
-    rt5.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_container",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList list3 = new TimelineFilterList();
-    list3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    list3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList list4 = new TimelineFilterList();
-    list4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    list4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto2"))));
-    TimelineFilterList combinedList =
-        new TimelineFilterList(Operator.OR, list3, list4);
-    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
-        new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto3"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-  }
-
-  @Test
-  public void testReadEntitiesDefaultView() throws Exception {
-    TimelineEntity e1 = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve());
-    assertNotNull(e1);
-    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
-        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
-        e1.getRelatesToEntities().isEmpty());
-    Set<TimelineEntity> es1 = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve());
-    assertEquals(3, es1.size());
-    for (TimelineEntity e : es1) {
-      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
-          e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
-          e.getRelatesToEntities().isEmpty());
-    }
-  }
-
-  @Test
-  public void testReadEntitiesByFields() throws Exception {
-    TimelineEntity e1 = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
-    assertNotNull(e1);
-    assertEquals(3, e1.getConfigs().size());
-    assertEquals(0, e1.getIsRelatedToEntities().size());
-    Set<TimelineEntity> es1 = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
-    assertEquals(3, es1.size());
-    int metricsCnt = 0;
-    int isRelatedToCnt = 0;
-    int infoCnt = 0;
-    for (TimelineEntity entity : es1) {
-      metricsCnt += entity.getMetrics().size();
-      isRelatedToCnt += entity.getIsRelatedToEntities().size();
-      infoCnt += entity.getInfo().size();
-    }
-    assertEquals(0, infoCnt);
-    assertEquals(4, isRelatedToCnt);
-    assertEquals(3, metricsCnt);
-  }
-
-  @Test
-  public void testReadEntitiesConfigPrefix() throws Exception {
-    TimelineFilterList list =
-        new TimelineFilterList(Operator.OR,
-            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
-    TimelineEntity e1 = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve(list, null, null, null));
-    assertNotNull(e1);
-    assertEquals(1, e1.getConfigs().size());
-    Set<TimelineEntity> es1 = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(list, null, null, null));
-    int cfgCnt = 0;
-    for (TimelineEntity entity : es1) {
-      cfgCnt += entity.getConfigs().size();
-      for (String confKey : entity.getConfigs().keySet()) {
-        assertTrue("Config key returned should start with cfg_",
-            confKey.startsWith("cfg_"));
-      }
-    }
-    assertEquals(3, cfgCnt);
-  }
-
-  @Test
-  public void testReadEntitiesConfigFilters() throws Exception {
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
-    list1.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
-    list2.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "config_param2", "value2"));
-    TimelineFilterList confFilterList =
-        new TimelineFilterList(Operator.OR, list1, list2);
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
-    assertEquals(2, entities.size());
-    int c

<TRUNCATED>

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


[11/50] [abbrv] hadoop git commit: YARN-5172. Update yarn daemonlog documentation due to HADOOP-12847. Contributed by Wei-Chiu Chuang

Posted by vi...@apache.org.
YARN-5172. Update yarn daemonlog documentation due to HADOOP-12847. Contributed by Wei-Chiu Chuang


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

Branch: refs/heads/HDFS-9806
Commit: b4a8fbcbbc5ea4ab3087ecf913839a53f32be113
Parents: 6fbfb50
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Oct 27 19:41:43 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Oct 27 19:43:02 2016 +0000

----------------------------------------------------------------------
 .../src/site/markdown/YarnCommands.md              | 17 ++---------------
 1 file changed, 2 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4a8fbcb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
index 2c38967..8f954ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
@@ -173,21 +173,8 @@ Commands useful for administrators of a Hadoop cluster.
 
 ### `daemonlog`
 
-Usage:
-
-```
-   yarn daemonlog -getlevel <host:httpport> <classname> 
-   yarn daemonlog -setlevel <host:httpport> <classname> <level>
-```
-
-| COMMAND\_OPTIONS | Description |
-|:---- |:---- |
-| -getlevel `<host:httpport>` `<classname>` | Prints the log level of the log identified by a qualified `<classname>`, in the daemon running at `<host:httpport>`. This command internally connects to `http://<host:httpport>/logLevel?log=<classname>` |
-| -setlevel `<host:httpport> <classname> <level>` | Sets the log level of the log identified by a qualified `<classname>` in the daemon running at `<host:httpport>`. This command internally connects to `http://<host:httpport>/logLevel?log=<classname>&level=<level>` |
-
-Get/Set the log level for a Log identified by a qualified class name in the daemon.
-
-Example: `$ bin/yarn daemonlog -setlevel 127.0.0.1:8088 org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl DEBUG`
+Get/Set the log level for a Log identified by a qualified class name in the daemon dynamically.
+See the Hadoop [Commands Manual](../../hadoop-project-dist/hadoop-common/CommandsManual.html#daemonlog) for more information.
 
 ### `nodemanager`
 


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


[46/50] [abbrv] hadoop git commit: YARN-5794. Fix the asflicense warnings. Contributed by Yiqun Lin

Posted by vi...@apache.org.
YARN-5794. Fix the asflicense warnings. Contributed by Yiqun Lin


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

Branch: refs/heads/HDFS-9806
Commit: 3de130252dd81b45b051f8aa01d281983ca7e541
Parents: 8fa2195
Author: Naganarasimha <na...@apache.org>
Authored: Fri Oct 28 12:37:37 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Fri Oct 28 12:37:37 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3de13025/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 f1be098..b2d3f32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -228,6 +228,10 @@
             <exclude>src/main/resources/webapps/static/dt-1.9.4/css/jui-dt.css</exclude>
             <exclude>src/main/resources/webapps/static/dt-1.9.4/css/demo_table.css</exclude>
             <exclude>src/main/resources/webapps/static/dt-1.9.4/images/Sorting icons.psd</exclude>
+            <exclude>src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js</exclude>
+            <exclude>src/main/resources/webapps/static/jt/jquery.jstree.js</exclude>
+            <exclude>src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js</exclude>
+            <exclude>src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js</exclude>
             <exclude>src/main/resources/webapps/static/jquery/themes-1.9.1/base/jquery-ui.css</exclude>
             <exclude>src/test/resources/application_1440536969523_0001.har/_index</exclude>
             <exclude>src/test/resources/application_1440536969523_0001.har/part-0</exclude>


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


[10/50] [abbrv] hadoop git commit: YARN-4456. Clean up Lint warnings in nodemanager (templedf via rkanter)

Posted by vi...@apache.org.
YARN-4456. Clean up Lint warnings in nodemanager (templedf via rkanter)


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

Branch: refs/heads/HDFS-9806
Commit: 6fbfb501f2e27045da5ce8f48dde881b29328b4a
Parents: ae48c49
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Oct 27 12:37:01 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Oct 27 12:37:01 2016 -0700

----------------------------------------------------------------------
 .../containermanager/logaggregation/TestLogAggregationService.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fbfb501/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index 1edb841..b9d18a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -2138,7 +2138,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
             ApplicationEventType.APPLICATION_LOG_HANDLING_INITED) };
     checkEvents(appEventHandler, expectedInitEvents, false, "getType",
         "getApplicationID");
-    reset(appEventHandler);
+    reset(new EventHandler[] {appEventHandler});
 
     logAggregationService.handle(new LogHandlerAppFinishedEvent(appId));
     logAggregationService.stop();


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


[17/50] [abbrv] hadoop git commit: YARN-3568. TestAMRMTokens should use some random port. (Takashi Ohnishi via Subru).

Posted by vi...@apache.org.
YARN-3568. TestAMRMTokens should use some random port. (Takashi Ohnishi via Subru).


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

Branch: refs/heads/HDFS-9806
Commit: 79ae78dcbec183ab53b26de408b4517e5a151878
Parents: b98fc82
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Oct 27 15:11:12 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Thu Oct 27 15:11:12 2016 -0700

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/security/TestAMRMTokens.java     | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79ae78dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
index 4488ad6..bcf239d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
@@ -113,6 +113,8 @@ public class TestAMRMTokens {
             DEFAULT_RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS);
     conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS,
         YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS);
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
+        "0.0.0.0:0");
 
     MyContainerManager containerManager = new MyContainerManager();
     final MockRMWithAMS rm =
@@ -230,6 +232,8 @@ public class TestAMRMTokens {
       YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
       rolling_interval_sec);
     conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, am_expire_ms);
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
+        "0.0.0.0:0");
     MyContainerManager containerManager = new MyContainerManager();
     final MockRMWithAMS rm =
         new MockRMWithAMS(conf, containerManager);


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