Browse Source

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

(cherry picked from commit b3c15e4ef763ebc4b033c686114fe627350824ac)
Naganarasimha 8 years ago
parent
commit
15f9c18ae4

+ 0 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java

@@ -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;
-  }
-
-}

+ 2 - 4
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

+ 2 - 2
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);

+ 4 - 12
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 =

+ 11 - 31
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 =

+ 4 - 8
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 =