Ver Fonte

YARN-5656. Fix ReservationACLsTestBase. (Sean Po via asuresh)

(cherry picked from commit 9f03b403ec69658fc57bc0f6b832da0e3c746497)
Arun Suresh há 8 anos atrás
pai
commit
2a9f809c96

+ 0 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.MismatchedUserException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
 
@@ -39,17 +38,6 @@ public class NoOverCommitPolicy implements SharingPolicy {
   public void validate(Plan plan, ReservationAllocation reservation)
       throws PlanningException {
 
-    ReservationAllocation oldReservation =
-        plan.getReservationById(reservation.getReservationId());
-
-    // check updates are using same name
-    if (oldReservation != null
-        && !oldReservation.getUser().equals(reservation.getUser())) {
-      throw new MismatchedUserException(
-          "Updating an existing reservation with mismatching user:"
-              + oldReservation.getUser() + " != " + reservation.getUser());
-    }
-
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
         reservation.getStartTime(), reservation.getEndTime());

+ 0 - 46
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.java

@@ -1,46 +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.reservation.exceptions;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-
-/**
- * Exception thrown when an update to an existing reservation is performed
- * by a user that is not the reservation owner. 
- */
-@Public
-@Unstable
-public class MismatchedUserException extends PlanningException {
-
-  private static final long serialVersionUID = 8313222590561668413L;
-
-  public MismatchedUserException(String message) {
-    super(message);
-  }
-
-  public MismatchedUserException(Throwable cause) {
-    super(cause);
-  }
-
-  public MismatchedUserException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-}

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java

@@ -566,6 +566,8 @@ public class ReservationACLsTestBase extends ACLsTestBase {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
+    out.println("  <defaultQueueSchedulingPolicy>drf" +
+        "</defaultQueueSchedulingPolicy>");
     out.println("  <queue name=\"queueA\">");
     out.println("    <aclSubmitReservations>" +
             "queueA_user,common_user " +

+ 0 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java

@@ -23,10 +23,8 @@ import static org.mockito.Mockito.mock;
 import java.io.IOException;
 
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
-import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.MismatchedUserException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -127,25 +125,6 @@ public class TestNoOverCommitPolicy {
             .generateAllocation(initTime, step, f), res, minAlloc), false);
   }
 
-  @Test(expected = MismatchedUserException.class)
-  public void testUserMismatch() throws IOException, PlanningException {
-    // generate allocation from single tenant that exceed capacity
-    int[] f = generateData(3600, (int) (0.5 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    ReservationId rid = ReservationSystemTestUtil.getNewReservationId();
-
-    plan.addReservation(new InMemoryReservationAllocation(rid, rDef, "u1",
-        "dedicated", initTime, initTime + f.length, ReservationSystemTestUtil
-            .generateAllocation(initTime, step, f), res, minAlloc), false);
-
-    // trying to update a reservation with a mismatching user
-    plan.updateReservation(new InMemoryReservationAllocation(rid, rDef, "u2",
-        "dedicated", initTime, initTime + f.length, ReservationSystemTestUtil
-            .generateAllocation(initTime, step, f), res, minAlloc));
-  }
-
   @Test
   public void testMultiTenantPass() throws IOException, PlanningException {
     // generate allocation from multiple tenants that barely fit in tot capacity