Browse Source

Reverting for fixing compilation errors in branch-2.
Revert "YARN-5649. Add REST endpoints for updating application timeouts. Contributed by Rohith Sharma K S"

This reverts commit 43796580421346f254d066c0c58ae13937cdbd45.

Rohith Sharma K S 8 years ago
parent
commit
87bb3c51d6
11 changed files with 6 additions and 472 deletions
  1. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  2. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
  3. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
  4. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
  5. 0 186
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
  6. 0 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
  7. 0 71
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppTimeoutInfo.java
  8. 0 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppTimeoutsInfo.java
  9. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
  10. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
  11. 1 136
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -1734,7 +1734,7 @@ public class ClientRMService extends AbstractService implements
       RMAuditLogger.logFailure(callerUGI.getShortUserName(),
           AuditConstants.UPDATE_APP_TIMEOUTS, "UNKNOWN", "ClientRMService",
           ex.getMessage());
-      throw ex;
+      throw RPCUtil.getRemoteException(ex);
     }
 
     RMAuditLogger.logSuccess(callerUGI.getShortUserName(),

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java

@@ -66,7 +66,6 @@ public class RMAuditLogger {
         "Update Application Priority";
     public static final String UPDATE_APP_TIMEOUTS =
         "Update Application Timeouts";
-    public static final String GET_APP_TIMEOUTS = "Get Application Timeouts";
     public static final String CHANGE_CONTAINER_RESOURCE =
         "AM Changed Container Resource";
     public static final String SIGNAL_CONTAINER = "Signal Container Request";

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java

@@ -515,7 +515,7 @@ public class RMServerUtils {
           String message =
               "Expire time is not in ISO8601 format. ISO8601 supported "
                   + "format is yyyy-MM-dd'T'HH:mm:ss.SSSZ";
-          throw new YarnException(message, ex);
+          throw new YarnException(message);
         }
         if (expireTime < currentTimeMillis) {
           String message =

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java

@@ -54,8 +54,7 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
             CapacitySchedulerQueueInfoList.class, ResourceInfo.class,
             UsersInfo.class, UserInfo.class, ApplicationStatisticsInfo.class,
             StatisticsItemInfo.class, CapacitySchedulerHealthInfo.class,
-            FairSchedulerQueueInfoList.class, AppTimeoutsInfo.class,
-            AppTimeoutInfo.class };
+            FairSchedulerQueueInfoList.class};
     // these dao classes need root unwrapping
     final Class[] rootUnwrappedTypes =
         { NewApplication.class, ApplicationSubmissionContextInfo.class,

+ 0 - 186
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java

@@ -24,11 +24,9 @@ import java.nio.ByteBuffer;
 import java.security.AccessControlException;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
-import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -97,12 +95,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -193,7 +189,6 @@ import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
 import org.apache.hadoop.yarn.util.AdHocLogDumper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
@@ -2393,185 +2388,4 @@ public class RMWebServices extends WebServices {
     return Response.status(Status.OK).entity(resResponse).build();
   }
 
-  @GET
-  @Path("/apps/{appid}/timeout/{type}")
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
-  public AppTimeoutInfo getAppTimeout(@Context HttpServletRequest hsr,
-      @PathParam("appid") String appId, @PathParam("type") String type)
-      throws AuthorizationException {
-    init();
-    RMApp app = validateAppTimeoutRequest(hsr, appId);
-
-    ApplicationTimeoutType appTimeoutType = parseTimeoutType(type);
-    Long timeoutValue = app.getApplicationTimeouts().get(appTimeoutType);
-    AppTimeoutInfo timeout =
-        constructAppTimeoutDao(appTimeoutType, timeoutValue);
-    return timeout;
-  }
-
-  private RMApp validateAppTimeoutRequest(HttpServletRequest hsr,
-      String appId) {
-    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
-    String userName = "UNKNOWN-USER";
-    if (callerUGI != null) {
-      userName = callerUGI.getUserName();
-    }
-
-    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
-      String msg = "The default static user cannot carry out this operation.";
-      RMAuditLogger.logFailure(userName, AuditConstants.GET_APP_TIMEOUTS,
-          "UNKNOWN", "RMWebService", msg);
-      throw new ForbiddenException(msg);
-    }
-
-    RMApp app = null;
-    try {
-      app = getRMAppForAppId(appId);
-    } catch (NotFoundException e) {
-      RMAuditLogger.logFailure(userName, AuditConstants.GET_APP_TIMEOUTS,
-          "UNKNOWN", "RMWebService",
-          "Trying to get timeouts of an absent application " + appId);
-      throw e;
-    }
-    return app;
-  }
-
-  @GET
-  @Path("/apps/{appid}/timeouts")
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
-  public AppTimeoutsInfo getAppTimeouts(@Context HttpServletRequest hsr,
-      @PathParam("appid") String appId) throws AuthorizationException {
-    init();
-
-    RMApp app = validateAppTimeoutRequest(hsr, appId);
-
-    AppTimeoutsInfo timeouts = new AppTimeoutsInfo();
-    Map<ApplicationTimeoutType, Long> applicationTimeouts =
-        app.getApplicationTimeouts();
-    if (applicationTimeouts.isEmpty()) {
-      // If application is not set timeout, lifetime should be sent as default
-      // with expiryTime=UNLIMITED and remainingTime=-1
-      timeouts
-          .add(constructAppTimeoutDao(ApplicationTimeoutType.LIFETIME, null));
-    } else {
-      for (Entry<ApplicationTimeoutType, Long> timeout : app
-          .getApplicationTimeouts().entrySet()) {
-        AppTimeoutInfo timeoutInfo =
-            constructAppTimeoutDao(timeout.getKey(), timeout.getValue());
-        timeouts.add(timeoutInfo);
-      }
-    }
-    return timeouts;
-  }
-
-  private ApplicationTimeoutType parseTimeoutType(String type) {
-    try {
-      // enum string is in the uppercase
-      return ApplicationTimeoutType
-          .valueOf(StringUtils.toUpperCase(type.trim()));
-    } catch (RuntimeException e) {
-      ApplicationTimeoutType[] typeArray = ApplicationTimeoutType.values();
-      String allAppTimeoutTypes = Arrays.toString(typeArray);
-      throw new BadRequestException("Invalid application-state " + type.trim()
-          + " specified. It should be one of " + allAppTimeoutTypes);
-    }
-  }
-
-  private AppTimeoutInfo constructAppTimeoutDao(ApplicationTimeoutType type,
-      Long timeoutInMillis) {
-    AppTimeoutInfo timeout = new AppTimeoutInfo();
-    timeout.setTimeoutType(type);
-    if (timeoutInMillis != null) {
-      timeout.setExpiryTime(Times.formatISO8601(timeoutInMillis.longValue()));
-      timeout.setRemainingTime(
-          Math.max((timeoutInMillis - System.currentTimeMillis()) / 1000, 0));
-    }
-    return timeout;
-  }
-
-  @PUT
-  @Path("/apps/{appid}/timeout")
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
-  @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
-      @Context HttpServletRequest hsr, @PathParam("appid") String appId)
-      throws AuthorizationException, YarnException, InterruptedException,
-      IOException {
-    init();
-
-    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
-    if (callerUGI == null) {
-      throw new AuthorizationException(
-          "Unable to obtain user name, user not authenticated");
-    }
-
-    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
-      return Response.status(Status.FORBIDDEN)
-          .entity("The default static user cannot carry out this operation.")
-          .build();
-    }
-
-    String userName = callerUGI.getUserName();
-    RMApp app = null;
-    try {
-      app = getRMAppForAppId(appId);
-    } catch (NotFoundException e) {
-      RMAuditLogger.logFailure(userName, AuditConstants.UPDATE_APP_TIMEOUTS,
-          "UNKNOWN", "RMWebService",
-          "Trying to update timeout of an absent application " + appId);
-      throw e;
-    }
-
-    return updateApplicationTimeouts(app, callerUGI, appTimeout);
-  }
-
-  private Response updateApplicationTimeouts(final RMApp app,
-      UserGroupInformation callerUGI, final AppTimeoutInfo appTimeout)
-      throws IOException, InterruptedException {
-
-    if (appTimeout.getTimeoutType() == null) {
-      return Response.status(Status.BAD_REQUEST).entity("Timeout type is null.")
-          .build();
-    }
-
-    String userName = callerUGI.getUserName();
-    try {
-      callerUGI.doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws IOException, YarnException {
-          UpdateApplicationTimeoutsRequest request =
-              UpdateApplicationTimeoutsRequest
-                  .newInstance(app.getApplicationId(), Collections.singletonMap(
-                      appTimeout.getTimeoutType(), appTimeout.getExpireTime()));
-          rm.getClientRMService().updateApplicationTimeouts(request);
-          return null;
-        }
-      });
-    } catch (UndeclaredThrowableException ue) {
-      // if the root cause is a permissions issue
-      // bubble that up to the user
-      if (ue.getCause() instanceof YarnException) {
-        YarnException ye = (YarnException) ue.getCause();
-        if (ye.getCause() instanceof AccessControlException) {
-          String appId = app.getApplicationId().toString();
-          String msg = "Unauthorized attempt to change timeout of app " + appId
-              + " by remote user " + userName;
-          return Response.status(Status.FORBIDDEN).entity(msg).build();
-        } else if (ye.getCause() instanceof ParseException) {
-          return Response.status(Status.BAD_REQUEST)
-              .entity(ye.getMessage()).build();
-        } else {
-          throw ue;
-        }
-      } else {
-        throw ue;
-      }
-    }
-    AppTimeoutInfo timeout = constructAppTimeoutDao(appTimeout.getTimeoutType(),
-        app.getApplicationTimeouts().get(appTimeout.getTimeoutType()));
-    return Response.status(Status.OK).entity(timeout).build();
-  }
 }

+ 0 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -30,7 +29,6 @@ import javax.xml.bind.annotation.XmlTransient;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
@@ -116,7 +114,6 @@ public class AppInfo {
   protected String amNodeLabelExpression;
 
   protected ResourcesInfo resourceInfo = null;
-  protected AppTimeoutsInfo timeouts = new AppTimeoutsInfo();
 
   public AppInfo() {
   } // JAXB needs this
@@ -243,27 +240,6 @@ public class AppInfo {
               : null;
         }
       }
-
-      Map<ApplicationTimeoutType, Long> applicationTimeouts =
-          app.getApplicationTimeouts();
-      if (applicationTimeouts.isEmpty()) {
-        // If application is not set timeout, lifetime should be sent as default
-        // with expiryTime=UNLIMITED and remainingTime=-1
-        AppTimeoutInfo timeoutInfo = new AppTimeoutInfo();
-        timeoutInfo.setTimeoutType(ApplicationTimeoutType.LIFETIME);
-        timeouts.add(timeoutInfo);
-      } else {
-        for (Map.Entry<ApplicationTimeoutType, Long> entry : app
-            .getApplicationTimeouts().entrySet()) {
-          AppTimeoutInfo timeout = new AppTimeoutInfo();
-          timeout.setTimeoutType(entry.getKey());
-          long timeoutInMillis = entry.getValue().longValue();
-          timeout.setExpiryTime(Times.formatISO8601(timeoutInMillis));
-          timeout.setRemainingTime(Math
-              .max((timeoutInMillis - System.currentTimeMillis()) / 1000, 0));
-          timeouts.add(timeout);
-        }
-      }
     }
   }
 

+ 0 - 71
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppTimeoutInfo.java

@@ -1,71 +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.webapp.dao;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
-
-/**
- * DAO object to display Application timeout information.
- */
-@XmlRootElement(name = "timeout")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class AppTimeoutInfo {
-
-  @XmlElement(name = "type")
-  private ApplicationTimeoutType timeoutType;
-
-  @XmlElement(name = "expiryTime")
-  private String expiryTime;
-
-  @XmlElement(name = "remainingTimeInSeconds")
-  private long remainingTimeInSec;
-
-  public AppTimeoutInfo() {
-    expiryTime = "UNLIMITED";
-    remainingTimeInSec = -1;
-  }
-
-  public ApplicationTimeoutType getTimeoutType() {
-    return timeoutType;
-  }
-
-  public String getExpireTime() {
-    return expiryTime;
-  }
-
-  public long getRemainingTimeInSec() {
-    return remainingTimeInSec;
-  }
-
-  public void setTimeoutType(ApplicationTimeoutType type) {
-    this.timeoutType = type;
-  }
-
-  public void setExpiryTime(String expiryTime) {
-    this.expiryTime = expiryTime;
-  }
-
-  public void setRemainingTime(long remainingTime) {
-    this.remainingTimeInSec = remainingTime;
-  }
-}

+ 0 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppTimeoutsInfo.java

@@ -1,47 +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.webapp.dao;
-
-import java.util.ArrayList;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * This class hosts a set of AppTimeout DAO objects.
- */
-@XmlRootElement(name = "timeouts")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class AppTimeoutsInfo {
-
-  @XmlElement(name = "timeout")
-  private ArrayList<AppTimeoutInfo> timeouts = new ArrayList<AppTimeoutInfo>();
-
-  public AppTimeoutsInfo() {
-  } // JAXB needs this
-
-  public void add(AppTimeoutInfo timeoutInfo) {
-    timeouts.add(timeoutInfo);
-  }
-
-  public ArrayList<AppTimeoutInfo> getAppTimeouts() {
-    return timeouts;
-  }
-}

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
 import java.util.Collection;
-import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
@@ -306,7 +305,7 @@ public class MockRMApp implements RMApp {
 
   @Override
   public Map<ApplicationTimeoutType, Long> getApplicationTimeouts() {
-    return Collections.emptyMap();
+    throw new UnsupportedOperationException("Not supported yet.");
   }
 
   @Override

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java

@@ -1370,7 +1370,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
   public void verifyAppInfo(JSONObject info, RMApp app) throws JSONException,
       Exception {
 
-    int expectedNumberOfElements = 35;
+    int expectedNumberOfElements = 34;
     String appNodeLabelExpression = null;
     String amNodeLabelExpression = null;
     if (app.getApplicationSubmissionContext()

+ 1 - 136
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java

@@ -59,7 +59,6 @@ import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHand
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -82,16 +81,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CredentialsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LogAggregationContextInfo;
-import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
-import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.After;
@@ -118,7 +115,6 @@ import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.client.filter.LoggingFilter;
-import com.sun.jersey.api.json.JSONConfiguration;
 import com.sun.jersey.api.json.JSONJAXBContext;
 import com.sun.jersey.api.json.JSONMarshaller;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@@ -1286,135 +1282,4 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     assertEquals(queue, responseQueue);
   }
 
-  @Test(timeout = 90000)
-  public void testUpdateAppTimeout() throws Exception {
-    client().addFilter(new LoggingFilter(System.out));
-
-    rm.start();
-    rm.registerNode("127.0.0.1:1234", 2048);
-    String[] mediaTypes =
-        { MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML };
-    MediaType[] contentTypes =
-        { MediaType.APPLICATION_JSON_TYPE, MediaType.APPLICATION_XML_TYPE };
-    for (String mediaType : mediaTypes) {
-      for (MediaType contentType : contentTypes) {
-        // application submitted without timeout
-        RMApp app = rm.submitApp(CONTAINER_MB, "", webserviceUserName);
-
-        ClientResponse response =
-            this.constructWebResource("apps", app.getApplicationId().toString(),
-                "timeouts").accept(mediaType).get(ClientResponse.class);
-        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
-          assertEquals(
-              MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
-              response.getType().toString());
-          JSONObject js =
-              response.getEntity(JSONObject.class).getJSONObject("timeouts");
-          JSONArray entity = js.getJSONArray("timeout");
-          verifyAppTimeoutJson(entity.getJSONObject(0),
-              ApplicationTimeoutType.LIFETIME, "UNLIMITED", -1);
-        }
-
-        AppTimeoutInfo timeoutUpdate = new AppTimeoutInfo();
-        long timeOutFromNow = 60;
-        String expireTime = Times
-            .formatISO8601(System.currentTimeMillis() + timeOutFromNow * 1000);
-        timeoutUpdate.setTimeoutType(ApplicationTimeoutType.LIFETIME);
-        timeoutUpdate.setExpiryTime(expireTime);
-
-        Object entity;
-        if (contentType.equals(MediaType.APPLICATION_JSON_TYPE)) {
-          entity = appTimeoutToJSON(timeoutUpdate);
-        } else {
-          entity = timeoutUpdate;
-        }
-        response = this
-            .constructWebResource("apps", app.getApplicationId().toString(),
-                "timeout")
-            .entity(entity, contentType).accept(mediaType)
-            .put(ClientResponse.class);
-
-        if (!isAuthenticationEnabled()) {
-          assertResponseStatusCode(Status.UNAUTHORIZED,
-              response.getStatusInfo());
-          continue;
-        }
-        assertResponseStatusCode(Status.OK, response.getStatusInfo());
-        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
-          verifyAppTimeoutJson(response, ApplicationTimeoutType.LIFETIME,
-              expireTime, timeOutFromNow);
-        } else {
-          verifyAppTimeoutXML(response, ApplicationTimeoutType.LIFETIME,
-              expireTime, timeOutFromNow);
-        }
-
-        // invoke get
-        response =
-            this.constructWebResource("apps", app.getApplicationId().toString(),
-                "timeout", ApplicationTimeoutType.LIFETIME.toString())
-                .accept(mediaType).get(ClientResponse.class);
-        assertResponseStatusCode(Status.OK, response.getStatusInfo());
-        if (mediaType.contains(MediaType.APPLICATION_JSON)) {
-          verifyAppTimeoutJson(response, ApplicationTimeoutType.LIFETIME,
-              expireTime, timeOutFromNow);
-        }
-      }
-    }
-    rm.stop();
-  }
-
-  protected static void verifyAppTimeoutJson(ClientResponse response,
-      ApplicationTimeoutType type, String expireTime, long timeOutFromNow)
-      throws JSONException {
-    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
-        response.getType().toString());
-    JSONObject jsonTimeout = response.getEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, jsonTimeout.length());
-    JSONObject json = jsonTimeout.getJSONObject("timeout");
-    verifyAppTimeoutJson(json, type, expireTime, timeOutFromNow);
-  }
-
-  protected static void verifyAppTimeoutJson(JSONObject json,
-      ApplicationTimeoutType type, String expireTime, long timeOutFromNow)
-      throws JSONException {
-    assertEquals("incorrect number of elements", 3, json.length());
-    assertEquals(type.toString(), json.getString("type"));
-    assertEquals(expireTime, json.getString("expiryTime"));
-    assertTrue(json.getLong("remainingTimeInSeconds") <= timeOutFromNow);
-  }
-
-  protected static void verifyAppTimeoutXML(ClientResponse response,
-      ApplicationTimeoutType type, String expireTime, long timeOutFromNow)
-      throws ParserConfigurationException, IOException, SAXException {
-    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();
-    InputSource is = new InputSource();
-    is.setCharacterStream(new StringReader(xml));
-    Document dom = db.parse(is);
-    NodeList nodes = dom.getElementsByTagName("timeout");
-    assertEquals("incorrect number of elements", 1, nodes.getLength());
-    Element element = (Element) nodes.item(0);
-    assertEquals(type.toString(),
-        WebServicesTestUtils.getXmlString(element, "type"));
-    assertEquals(expireTime,
-        WebServicesTestUtils.getXmlString(element, "expiryTime"));
-    assertTrue(WebServicesTestUtils.getXmlLong(element,
-        "remainingTimeInSeconds") < timeOutFromNow);
-  }
-
-  protected static String appTimeoutToJSON(AppTimeoutInfo timeout)
-      throws Exception {
-    StringWriter sw = new StringWriter();
-    JSONJAXBContext ctx = new JSONJAXBContext(
-        JSONConfiguration.natural().rootUnwrapping(false).build(),
-        AppTimeoutInfo.class);
-    JSONMarshaller jm = ctx.createJSONMarshaller();
-    jm.marshallToJSON(timeout, sw);
-    jm.marshallToJSON(timeout, System.out);
-    return sw.toString();
-  }
-
 }