YARN-5221. Expose UpdateResourceRequest API to allow AM to request for change in container properties. (asuresh)

(cherry picked from commit d6d9cff21b)
This commit is contained in:
Arun Suresh 2016-08-27 15:22:43 -07:00
parent 56c9a96a76
commit b279f42d79
74 changed files with 2091 additions and 1095 deletions

View File

@ -59,6 +59,7 @@
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.client.ClientRMProxy;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
@ -296,8 +297,7 @@ public AllocateResponse allocate(AllocateRequest request)
Resources.none(), null, 1, null,
Collections.<NMToken>emptyList(),
yarnToken,
Collections.<Container>emptyList(),
Collections.<Container>emptyList());
Collections.<UpdatedContainer>emptyList());
response.setApplicationPriority(Priority.newInstance(0));
return response;
}

View File

@ -97,7 +97,6 @@
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NMToken;
@ -106,6 +105,7 @@
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
@ -1701,8 +1701,8 @@ public synchronized Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests) {
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
for (ResourceRequest req : ask) {
ResourceRequest reqCopy = ResourceRequest.newInstance(req
@ -1748,8 +1748,8 @@ public synchronized Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequest,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequest,
List<UpdateContainerRequest> decreaseRequests) {
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
for (ResourceRequest req : ask) {
ResourceRequest reqCopy = ResourceRequest.newInstance(req

View File

@ -51,7 +51,6 @@
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
@ -60,6 +59,7 @@
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@ -207,8 +207,8 @@ public void run() {
public Allocation allocate(ApplicationAttemptId attemptId,
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
List<String> strings, List<String> strings2,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests) {
if (metricsON) {
final Timer.Context context = schedulerAllocateTimer.time();
Allocation allocation = null;

View File

@ -27,8 +27,8 @@
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.util.Records;
/**
@ -48,13 +48,8 @@
* A list of unused {@link Container} which are being returned.
* </li>
* <li>
* A list of {@link ContainerResourceChangeRequest} to inform
* the <code>ResourceManager</code> about the resource increase
* requirements of running containers.
* </li>
* <li>
* A list of {@link ContainerResourceChangeRequest} to inform
* the <code>ResourceManager</code> about the resource decrease
* A list of {@link UpdateContainerRequest} to inform
* the <code>ResourceManager</code> about the change in
* requirements of running containers.
* </li>
* </ul>
@ -72,25 +67,23 @@ public static AllocateRequest newInstance(int responseID, float appProgress,
List<ContainerId> containersToBeReleased,
ResourceBlacklistRequest resourceBlacklistRequest) {
return newInstance(responseID, appProgress, resourceAsk,
containersToBeReleased, resourceBlacklistRequest, null, null);
containersToBeReleased, resourceBlacklistRequest, null);
}
@Public
@Stable
@Unstable
public static AllocateRequest newInstance(int responseID, float appProgress,
List<ResourceRequest> resourceAsk,
List<ContainerId> containersToBeReleased,
ResourceBlacklistRequest resourceBlacklistRequest,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> updateRequests) {
AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
allocateRequest.setResponseId(responseID);
allocateRequest.setProgress(appProgress);
allocateRequest.setAskList(resourceAsk);
allocateRequest.setReleaseList(containersToBeReleased);
allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest);
allocateRequest.setIncreaseRequests(increaseRequests);
allocateRequest.setDecreaseRequests(decreaseRequests);
allocateRequest.setUpdateRequests(updateRequests);
return allocateRequest;
}
@ -197,48 +190,25 @@ public abstract void setResourceBlacklistRequest(
ResourceBlacklistRequest resourceBlacklistRequest);
/**
* Get the list of container resource increase requests being sent by the
* Get the list of container update requests being sent by the
* <code>ApplicationMaster</code>.
* @return the list of {@link ContainerResourceChangeRequest}
* @return list of {@link UpdateContainerRequest}
* being sent by the
* <code>ApplicationMaster</code>.
*/
@Public
@Unstable
public abstract List<ContainerResourceChangeRequest> getIncreaseRequests();
public abstract List<UpdateContainerRequest> getUpdateRequests();
/**
* Set the list of container resource increase requests to inform the
* <code>ResourceManager</code> about the containers whose resources need
* to be increased.
* @param increaseRequests list of
* {@link ContainerResourceChangeRequest}
* Set the list of container update requests to inform the
* <code>ResourceManager</code> about the containers that need to be
* updated.
* @param updateRequests list of <code>UpdateContainerRequest</code> for
* containers to be updated
*/
@Public
@Unstable
public abstract void setIncreaseRequests(
List<ContainerResourceChangeRequest> increaseRequests);
/**
* Get the list of container resource decrease requests being sent by the
* <code>ApplicationMaster</code>.
* @return list of {@link ContainerResourceChangeRequest}
* being sent by the
* <code>ApplicationMaster</code>.
*/
@Public
@Unstable
public abstract List<ContainerResourceChangeRequest> getDecreaseRequests();
/**
* Set the list of container resource decrease requests to inform the
* <code>ResourceManager</code> about the containers whose resources need
* to be decreased.
* @param decreaseRequests list of
* {@link ContainerResourceChangeRequest}
*/
@Public
@Unstable
public abstract void setDecreaseRequests(
List<ContainerResourceChangeRequest> decreaseRequests);
public abstract void setUpdateRequests(
List<UpdateContainerRequest> updateRequests);
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.api.protocolrecords;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -35,6 +36,8 @@
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.util.Records;
/**
@ -95,19 +98,17 @@ public static AllocateResponse newInstance(int responseId,
}
@Public
@Stable
@Unstable
public static AllocateResponse newInstance(int responseId,
List<ContainerStatus> completedContainers,
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
Resource availResources, AMCommand command, int numClusterNodes,
PreemptionMessage preempt, List<NMToken> nmTokens,
List<Container> increasedContainers,
List<Container> decreasedContainers) {
List<UpdatedContainer> updatedContainers) {
AllocateResponse response = newInstance(responseId, completedContainers,
allocatedContainers, updatedNodes, availResources, command,
numClusterNodes, preempt, nmTokens);
response.setIncreasedContainers(increasedContainers);
response.setDecreasedContainers(decreasedContainers);
response.setUpdatedContainers(updatedContainers);
return response;
}
@ -118,12 +119,11 @@ public static AllocateResponse newInstance(int responseId,
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
Resource availResources, AMCommand command, int numClusterNodes,
PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
List<Container> increasedContainers,
List<Container> decreasedContainers) {
List<UpdatedContainer> updatedContainers) {
AllocateResponse response =
newInstance(responseId, completedContainers, allocatedContainers,
updatedNodes, availResources, command, numClusterNodes, preempt,
nmTokens, increasedContainers, decreasedContainers);
nmTokens, updatedContainers);
response.setAMRMToken(amRMToken);
return response;
}
@ -271,40 +271,24 @@ public static AllocateResponse newInstance(int responseId,
public abstract void setNMTokens(List<NMToken> nmTokens);
/**
* Get the list of newly increased containers by
* Get the list of newly updated containers by
* <code>ResourceManager</code>.
* @return list of newly increased containers
*/
@Public
@Unstable
public abstract List<Container> getIncreasedContainers();
public abstract List<UpdatedContainer> getUpdatedContainers();
/**
* Set the list of newly increased containers by
* Set the list of newly updated containers by
* <code>ResourceManager</code>.
*
* @param updatedContainers List of Updated Containers.
*/
@Private
@Unstable
public abstract void setIncreasedContainers(
List<Container> increasedContainers);
/**
* Get the list of newly decreased containers by
* <code>ResourceManager</code>.
* @return the list of newly decreased containers
*/
@Public
@Unstable
public abstract List<Container> getDecreasedContainers();
/**
* Set the list of newly decreased containers by
* <code>ResourceManager</code>.
*/
@Private
@Unstable
public abstract void setDecreasedContainers(
List<Container> decreasedContainers);
public abstract void setUpdatedContainers(
List<UpdatedContainer> updatedContainers);
/**
* The AMRMToken that belong to this attempt
@ -331,4 +315,29 @@ public abstract void setDecreasedContainers(
@Private
@Unstable
public abstract void setApplicationPriority(Priority priority);
/**
* Get the list of container update errors to inform the
* Application Master about the container updates that could not be
* satisfied due to error.
*
* @return List of Update Container Errors.
*/
@Public
@Unstable
public List<UpdateContainerError> getUpdateErrors() {
return new ArrayList<>();
}
/**
* Set the list of container update errors to inform the
* Application Master about the container updates that could not be
* satisfied due to error.
* @param updateErrors list of <code>UpdateContainerError</code> for
* containers updates requests that were in error
*/
@Public
@Unstable
public void setUpdateErrors(List<UpdateContainerError> updateErrors) {
}
}

View File

@ -230,8 +230,30 @@ public long getAllocationRequestId() {
* allocation.
*/
@Private
@Evolving
@Unstable
public void setAllocationRequestId(long allocationRequestID) {
throw new UnsupportedOperationException();
}
/**
* Get the version of this container. The version will be incremented when
* a container is updated.
*
* @return version of this container.
*/
@Private
@Unstable
public int getVersion() {
return 0;
}
/**
* Set the version of this container.
* @param version of this container.
*/
@Private
@Unstable
public void setVersion(int version) {
throw new UnsupportedOperationException();
}
}

View File

@ -1,117 +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.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.util.Records;
/**
* {@code ContainerResourceChangeRequest} represents the request made by an
* application to the {@code ResourceManager} to change resource allocation of
* a running {@code Container}.
* <p>
* It includes:
* <ul>
* <li>{@link ContainerId} for the container.</li>
* <li>
* {@link Resource} capability of the container after the resource change
* is completed.
* </li>
* </ul>
*
* @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/
@Public
@Unstable
public abstract class ContainerResourceChangeRequest {
@Public
@Unstable
public static ContainerResourceChangeRequest newInstance(
ContainerId existingContainerId, Resource targetCapability) {
ContainerResourceChangeRequest context = Records
.newRecord(ContainerResourceChangeRequest.class);
context.setContainerId(existingContainerId);
context.setCapability(targetCapability);
return context;
}
/**
* Get the <code>ContainerId</code> of the container.
* @return <code>ContainerId</code> of the container
*/
@Public
@Unstable
public abstract ContainerId getContainerId();
/**
* Set the <code>ContainerId</code> of the container.
* @param containerId <code>ContainerId</code> of the container
*/
@Public
@Unstable
public abstract void setContainerId(ContainerId containerId);
/**
* Get the <code>Resource</code> capability of the container.
* @return <code>Resource</code> capability of the container
*/
@Public
@Unstable
public abstract Resource getCapability();
/**
* Set the <code>Resource</code> capability of the container.
* @param capability <code>Resource</code> capability of the container
*/
@Public
@Unstable
public abstract void setCapability(Resource capability);
@Override
public int hashCode() {
return getCapability().hashCode() + getContainerId().hashCode();
}
@Override
public boolean equals(Object other) {
if (other instanceof ContainerResourceChangeRequest) {
ContainerResourceChangeRequest ctx =
(ContainerResourceChangeRequest) other;
if (getContainerId() == null && ctx.getContainerId() != null) {
return false;
} else if (!getContainerId().equals(ctx.getContainerId())) {
return false;
}
if (getCapability() == null && ctx.getCapability() != null) {
return false;
} else if (!getCapability().equals(ctx.getCapability())) {
return false;
}
return true;
} else {
return false;
}
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.api.records;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Encodes the type of Container Update.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public enum ContainerUpdateType {
/**
* Resource increase.
*/
INCREASE_RESOURCE,
/**
* Resource decrease.
*/
DECREASE_RESOURCE,
/**
* Execution Type change.
*/
UPDATE_EXECUTION_TYPE
}

View File

@ -0,0 +1,119 @@
/**
* 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.api.records;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.util.Records;
/**
* {@code UpdateContainerError} is used by the Scheduler to notify the
* ApplicationMaster of an UpdateContainerRequest it cannot satisfy due to
* an error in the request. It includes the update request as well as
* a reason for why the request was not satisfiable.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract class UpdateContainerError {
@InterfaceAudience.Public
@InterfaceStability.Unstable
public static UpdateContainerError newInstance(String reason,
UpdateContainerRequest updateContainerRequest) {
UpdateContainerError error = Records.newRecord(UpdateContainerError.class);
error.setReason(reason);
error.setUpdateContainerRequest(updateContainerRequest);
return error;
}
/**
* Get reason why the update request was not satisfiable.
* @return Reason
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract String getReason();
/**
* Set reason why the update request was not satisfiable.
* @param reason Reason
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setReason(String reason);
/**
* Get the {@code UpdateContainerRequest} that was not satisfiable.
* @return UpdateContainerRequest
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract UpdateContainerRequest getUpdateContainerRequest();
/**
* Set the {@code UpdateContainerRequest} that was not satisfiable.
* @param updateContainerRequest Update Container Request
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setUpdateContainerRequest(
UpdateContainerRequest updateContainerRequest);
@Override
public int hashCode() {
final int prime = 2153;
int result = 2459;
String reason = getReason();
UpdateContainerRequest updateReq = getUpdateContainerRequest();
result = prime * result + ((reason == null) ? 0 : reason.hashCode());
result = prime * result + ((updateReq == null) ? 0 : updateReq.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
UpdateContainerError other = (UpdateContainerError) obj;
String reason = getReason();
if (reason == null) {
if (other.getReason() != null) {
return false;
}
} else if (!reason.equals(other.getReason())) {
return false;
}
UpdateContainerRequest req = getUpdateContainerRequest();
if (req == null) {
if (other.getUpdateContainerRequest() != null) {
return false;
}
} else if (!req.equals(other.getUpdateContainerRequest())) {
return false;
}
return true;
}
}

View File

@ -0,0 +1,218 @@
/**
* 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.api.records;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.util.Records;
/**
* {@code UpdateContainerRequest} represents the request made by an
* application to the {@code ResourceManager} to update an attribute of a
* {@code Container} such as its Resource allocation or (@code ExecutionType}
* <p>
* It includes:
* <ul>
* <li>version for the container.</li>
* <li>{@link ContainerId} for the container.</li>
* <li>
* {@link Resource} capability of the container after the update request
* is completed.
* </li>
* <li>
* {@link ExecutionType} of the container after the update request is
* completed.
* </li>
* </ul>
*
* Update rules:
* <ul>
* <li>
* Currently only ONE aspect of the container can be updated per request
* (user can either update Capability OR ExecutionType in one request..
* not both).
* </li>
* <li>
* There must be only 1 update request per container in an allocate call.
* </li>
* <li>
* If a new update request is sent for a container (in a subsequent allocate
* call) before the first one is satisfied by the Scheduler, it will
* overwrite the previous request.
* </li>
* </ul>
* @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract class UpdateContainerRequest {
@InterfaceAudience.Public
@InterfaceStability.Unstable
public static UpdateContainerRequest newInstance(int version,
ContainerId containerId, ContainerUpdateType updateType,
Resource targetCapability, ExecutionType targetExecutionType) {
UpdateContainerRequest request =
Records.newRecord(UpdateContainerRequest.class);
request.setContainerVersion(version);
request.setContainerId(containerId);
request.setContainerUpdateType(updateType);
request.setExecutionType(targetExecutionType);
request.setCapability(targetCapability);
return request;
}
/**
* Get the <code>ContainerId</code> of the container.
* @return <code>ContainerId</code> of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract int getContainerVersion();
/**
* Set the current version of the container.
* @param containerVersion of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setContainerVersion(int containerVersion);
/**
* Get the <code>ContainerUpdateType</code> of the container.
* @return <code>ContainerUpdateType</code> of the container.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract ContainerUpdateType getContainerUpdateType();
/**
* Set the <code>ContainerUpdateType</code> of the container.
* @param updateType of the Container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setContainerUpdateType(ContainerUpdateType updateType);
/**
* Get the <code>ContainerId</code> of the container.
* @return <code>ContainerId</code> of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract ContainerId getContainerId();
/**
* Set the <code>ContainerId</code> of the container.
* @param containerId <code>ContainerId</code> of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setContainerId(ContainerId containerId);
/**
* Get the <code>Resource</code> capability of the container.
* @return <code>Resource</code> capability of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract Resource getCapability();
/**
* Set the <code>Resource</code> capability of the container.
* @param capability <code>Resource</code> capability of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setCapability(Resource capability);
/**
* Get the target <code>ExecutionType</code> of the container.
* @return <code>ExecutionType</code> of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract ExecutionType getExecutionType();
/**
* Set the target <code>ExecutionType</code> of the container.
* @param executionType <code>ExecutionType</code> of the container
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract void setExecutionType(ExecutionType executionType);
@Override
public int hashCode() {
final int prime = 2153;
int result = 2459;
ContainerId cId = getContainerId();
ExecutionType execType = getExecutionType();
Resource capability = getCapability();
result =
prime * result + ((capability == null) ? 0 : capability.hashCode());
result = prime * result + ((cId == null) ? 0 : cId.hashCode());
result = prime * result + getContainerVersion();
result = prime * result + ((execType == null) ? 0 : execType.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
UpdateContainerRequest other = (UpdateContainerRequest) obj;
Resource capability = getCapability();
if (capability == null) {
if (other.getCapability() != null) {
return false;
}
} else if (!capability.equals(other.getCapability())) {
return false;
}
ContainerId cId = getContainerId();
if (cId == null) {
if (other.getContainerId() != null) {
return false;
}
} else if (!cId.equals(other.getContainerId())) {
return false;
}
if (getContainerVersion() != other.getContainerVersion()) {
return false;
}
ExecutionType execType = getExecutionType();
if (execType == null) {
if (other.getExecutionType() != null) {
return false;
}
} else if (!execType.equals(other.getExecutionType())) {
return false;
}
return true;
}
}

View File

@ -0,0 +1,118 @@
/**
* 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.api.records;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.util.Records;
/**
* An object that encapsulates an updated container and the
* type of Update.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract class UpdatedContainer {
/**
* Static Factory method.
*
* @param updateType ContainerUpdateType
* @param container Container
* @return UpdatedContainer
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public static UpdatedContainer newInstance(ContainerUpdateType updateType,
Container container) {
UpdatedContainer updatedContainer =
Records.newRecord(UpdatedContainer.class);
updatedContainer.setUpdateType(updateType);
updatedContainer.setContainer(container);
return updatedContainer;
}
/**
* Get the <code>ContainerUpdateType</code>.
* @return ContainerUpdateType
*/
public abstract ContainerUpdateType getUpdateType();
/**
* Set the <code>ContainerUpdateType</code>.
* @param updateType ContainerUpdateType
*/
public abstract void setUpdateType(ContainerUpdateType updateType);
/**
* Get the <code>Container</code>.
* @return Container
*/
public abstract Container getContainer();
/**
* Set the <code>Container</code>.
* @param container Container
*/
public abstract void setContainer(Container container);
@Override
public int hashCode() {
final int prime = 2153;
int result = 2459;
ContainerUpdateType updateType = getUpdateType();
Container container = getContainer();
result = prime * result + ((updateType == null) ? 0 :
updateType.hashCode());
result = prime * result + ((container == null) ? 0 : container.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
UpdatedContainer other = (UpdatedContainer) obj;
ContainerUpdateType updateType = getUpdateType();
if (updateType == null) {
if (other.getUpdateType() != null) {
return false;
}
} else if (updateType != other.getUpdateType()) {
return false;
}
Container container = getContainer();
if (container == null) {
if (other.getContainer() != null) {
return false;
}
} else if (!container.equals(other.getContainer())) {
return false;
}
return true;
}
}

View File

@ -94,6 +94,7 @@ message ContainerProto {
optional hadoop.common.TokenProto container_token = 6;
optional ExecutionTypeProto execution_type = 7 [default = GUARANTEED];
optional int64 allocation_request_id = 8 [default = -1];
optional int32 version = 9 [default = 0];
}
message ContainerReportProto {
@ -540,11 +541,6 @@ enum ContainerExitStatusProto {
DISKS_FAILED = -101;
}
message ContainerResourceChangeRequestProto {
optional ContainerIdProto container_id = 1;
optional ResourceProto capability = 2;
}
message ContainerRetryContextProto {
optional ContainerRetryPolicyProto retry_policy = 1 [default = NEVER_RETRY];
repeated int32 error_codes = 2;

View File

@ -60,14 +60,32 @@ message FinishApplicationMasterResponseProto {
optional bool isUnregistered = 1 [default = false];
}
enum ContainerUpdateTypeProto {
INCREASE_RESOURCE = 0;
DECREASE_RESOURCE = 1;
UPDATE_EXECUTION_TYPE = 2;
}
message UpdateContainerRequestProto {
required int32 container_version = 1;
required ContainerIdProto container_id = 2;
required ContainerUpdateTypeProto update_type = 3;
optional ResourceProto capability = 4;
optional ExecutionTypeProto execution_type = 5;
}
message UpdateContainerErrorProto {
optional string reason = 1;
optional UpdateContainerRequestProto update_request = 2;
}
message AllocateRequestProto {
repeated ResourceRequestProto ask = 1;
repeated ContainerIdProto release = 2;
optional ResourceBlacklistRequestProto blacklist_request = 3;
optional int32 response_id = 4;
optional float progress = 5;
repeated ContainerResourceChangeRequestProto increase_request = 6;
repeated ContainerResourceChangeRequestProto decrease_request = 7;
repeated UpdateContainerRequestProto update_requests = 6;
}
message NMTokenProto {
@ -75,6 +93,11 @@ message NMTokenProto {
optional hadoop.common.TokenProto token = 2;
}
message UpdatedContainerProto {
required ContainerUpdateTypeProto update_type = 1;
required ContainerProto container = 2;
}
message AllocateResponseProto {
optional AMCommandProto a_m_command = 1;
optional int32 response_id = 2;
@ -85,10 +108,10 @@ message AllocateResponseProto {
optional int32 num_cluster_nodes = 7;
optional PreemptionMessageProto preempt = 8;
repeated NMTokenProto nm_tokens = 9;
repeated ContainerProto increased_containers = 10;
repeated ContainerProto decreased_containers = 11;
repeated UpdatedContainerProto updated_containers = 10;
optional hadoop.common.TokenProto am_rm_token = 12;
optional PriorityProto application_priority = 13;
repeated UpdateContainerErrorProto update_errors = 15;
}
enum SchedulerResourceTypes {

View File

@ -90,6 +90,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
@ -880,7 +881,8 @@ public void onContainersAllocated(List<Container> allocatedContainers) {
}
@Override
public void onContainersResourceChanged(List<Container> containers) {}
public void onContainersUpdated(
List<UpdatedContainer> containers) {}
@Override
public void onShutdownRequest() {

View File

@ -29,6 +29,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.classification.InterfaceStability.Stable;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@ -40,6 +41,7 @@
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl;
@ -62,7 +64,7 @@
* [run tasks on the containers]
* }
*
* public void onContainersResourceChanged(List<Container> containers) {
* public void onContainersUpdated(List<Container> containers) {
* [determine if resource allocation of containers have been increased in
* the ResourceManager, and if so, inform the NodeManagers to increase the
* resource monitor/enforcement on the containers]
@ -409,8 +411,9 @@ public abstract static class AbstractCallbackHandler
* Called when the ResourceManager responds to a heartbeat with containers
* whose resource allocation has been changed.
*/
public abstract void onContainersResourceChanged(
List<Container> containers);
@Public
@Unstable
public abstract void onContainersUpdated(List<UpdatedContainer> containers);
/**
* Called when the ResourceManager wants the ApplicationMaster to shutdown

View File

@ -39,6 +39,7 @@
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
@ -338,12 +339,11 @@ public void run() {
if (handler instanceof AMRMClientAsync.AbstractCallbackHandler) {
// RM side of the implementation guarantees that there are
// no duplications between increased and decreased containers
List<Container> changed = new ArrayList<>();
changed.addAll(response.getIncreasedContainers());
changed.addAll(response.getDecreasedContainers());
List<UpdatedContainer> changed = new ArrayList<>();
changed.addAll(response.getUpdatedContainers());
if (!changed.isEmpty()) {
((AMRMClientAsync.AbstractCallbackHandler) handler)
.onContainersResourceChanged(changed);
.onContainersUpdated(changed);
}
}

View File

@ -52,8 +52,8 @@
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@ -63,6 +63,8 @@
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.client.ClientRMProxy;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@ -261,36 +263,10 @@ public AllocateResponse allocate(float progressIndicator)
new HashMap<>();
try {
synchronized (this) {
askList = new ArrayList<ResourceRequest>(ask.size());
for(ResourceRequest r : ask) {
// create a copy of ResourceRequest as we might change it while the
// RPC layer is using it to send info across
ResourceRequest rr = ResourceRequest.newInstance(r.getPriority(),
r.getResourceName(), r.getCapability(), r.getNumContainers(),
r.getRelaxLocality(), r.getNodeLabelExpression(),
r.getExecutionTypeRequest());
rr.setAllocationRequestId(r.getAllocationRequestId());
askList.add(rr);
}
List<ContainerResourceChangeRequest> increaseList = new ArrayList<>();
List<ContainerResourceChangeRequest> decreaseList = new ArrayList<>();
askList = cloneAsks();
// Save the current change for recovery
oldChange.putAll(change);
for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
change.entrySet()) {
Container container = entry.getValue().getKey();
Resource original = container.getResource();
Resource target = entry.getValue().getValue();
if (Resources.fitsIn(target, original)) {
// This is a decrease request
decreaseList.add(ContainerResourceChangeRequest.newInstance(
container.getId(), target));
} else {
// This is an increase request
increaseList.add(ContainerResourceChangeRequest.newInstance(
container.getId(), target));
}
}
List<UpdateContainerRequest> updateList = createUpdateList();
releaseList = new ArrayList<ContainerId>(release);
// optimistically clear this collection assuming no RPC failure
ask.clear();
@ -306,8 +282,7 @@ public AllocateResponse allocate(float progressIndicator)
allocateRequest =
AllocateRequest.newInstance(lastResponseId, progressIndicator,
askList, releaseList, blacklistRequest,
increaseList, decreaseList);
askList, releaseList, blacklistRequest, updateList);
// clear blacklistAdditions and blacklistRemovals before
// unsynchronized part
blacklistAdditions.clear();
@ -358,9 +333,8 @@ public AllocateResponse allocate(float progressIndicator)
if (!pendingChange.isEmpty()) {
List<ContainerStatus> completed =
allocateResponse.getCompletedContainersStatuses();
List<Container> changed = new ArrayList<>();
changed.addAll(allocateResponse.getIncreasedContainers());
changed.addAll(allocateResponse.getDecreasedContainers());
List<UpdatedContainer> changed = new ArrayList<>();
changed.addAll(allocateResponse.getUpdatedContainers());
// remove all pending change requests that belong to the completed
// containers
for (ContainerStatus status : completed) {
@ -417,6 +391,40 @@ public AllocateResponse allocate(float progressIndicator)
return allocateResponse;
}
private List<UpdateContainerRequest> createUpdateList() {
List<UpdateContainerRequest> updateList = new ArrayList<>();
for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
change.entrySet()) {
Resource targetCapability = entry.getValue().getValue();
Resource currCapability = entry.getValue().getKey().getResource();
int version = entry.getValue().getKey().getVersion();
ContainerUpdateType updateType =
ContainerUpdateType.INCREASE_RESOURCE;
if (Resources.fitsIn(targetCapability, currCapability)) {
updateType = ContainerUpdateType.DECREASE_RESOURCE;
}
updateList.add(
UpdateContainerRequest.newInstance(version, entry.getKey(),
updateType, targetCapability, null));
}
return updateList;
}
private List<ResourceRequest> cloneAsks() {
List<ResourceRequest> askList = new ArrayList<ResourceRequest>(ask.size());
for(ResourceRequest r : ask) {
// create a copy of ResourceRequest as we might change it while the
// RPC layer is using it to send info across
ResourceRequest rr = ResourceRequest.newInstance(r.getPriority(),
r.getResourceName(), r.getCapability(), r.getNumContainers(),
r.getRelaxLocality(), r.getNodeLabelExpression(),
r.getExecutionTypeRequest());
rr.setAllocationRequestId(r.getAllocationRequestId());
askList.add(rr);
}
return askList;
}
protected void removePendingReleaseRequests(
List<ContainerStatus> completedContainersStatuses) {
for (ContainerStatus containerStatus : completedContainersStatuses) {
@ -425,16 +433,16 @@ protected void removePendingReleaseRequests(
}
protected void removePendingChangeRequests(
List<Container> changedContainers) {
for (Container changedContainer : changedContainers) {
ContainerId containerId = changedContainer.getId();
List<UpdatedContainer> changedContainers) {
for (UpdatedContainer changedContainer : changedContainers) {
ContainerId containerId = changedContainer.getContainer().getId();
if (pendingChange.get(containerId) == null) {
continue;
}
if (LOG.isDebugEnabled()) {
LOG.debug("RM has confirmed changed resource allocation for "
+ "container " + containerId + ". Current resource allocation:"
+ changedContainer.getResource()
+ changedContainer.getContainer().getResource()
+ ". Remove pending change request:"
+ pendingChange.get(containerId).getValue());
}

View File

@ -45,9 +45,11 @@
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
@ -89,20 +91,21 @@ public void testAMRMClientAsync() throws Exception {
TestCallbackHandler callbackHandler = new TestCallbackHandler();
final AMRMClient<ContainerRequest> client = mock(AMRMClientImpl.class);
final AtomicInteger secondHeartbeatSync = new AtomicInteger(0);
when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(new Answer<AllocateResponse>() {
@Override
public AllocateResponse answer(InvocationOnMock invocation)
throws Throwable {
secondHeartbeatSync.incrementAndGet();
while (heartbeatBlock.get()) {
synchronized (heartbeatBlock) {
heartbeatBlock.wait();
when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(
new Answer<AllocateResponse>() {
@Override
public AllocateResponse answer(InvocationOnMock invocation)
throws Throwable {
secondHeartbeatSync.incrementAndGet();
while (heartbeatBlock.get()) {
synchronized (heartbeatBlock) {
heartbeatBlock.wait();
}
}
secondHeartbeatSync.incrementAndGet();
return response2;
}
}
secondHeartbeatSync.incrementAndGet();
return response2;
}
}).thenReturn(response3).thenReturn(emptyResponse);
}).thenReturn(response3).thenReturn(emptyResponse);
when(client.registerApplicationMaster(anyString(), anyInt(), anyString()))
.thenReturn(null);
when(client.getAvailableResources()).thenAnswer(new Answer<Resource>() {
@ -410,10 +413,21 @@ private AllocateResponse createAllocateResponse(
List<ContainerStatus> completed, List<Container> allocated,
List<Container> increased, List<Container> decreased,
List<NMToken> nmTokens) {
List<UpdatedContainer> updatedContainers = new ArrayList<>();
for (Container c : increased) {
updatedContainers.add(
UpdatedContainer.newInstance(
ContainerUpdateType.INCREASE_RESOURCE, c));
}
for (Container c : decreased) {
updatedContainers.add(
UpdatedContainer.newInstance(
ContainerUpdateType.DECREASE_RESOURCE, c));
}
AllocateResponse response =
AllocateResponse.newInstance(0, completed, allocated,
new ArrayList<NodeReport>(), null, null, 1, null, nmTokens,
increased, decreased);
updatedContainers);
return response;
}
@ -429,7 +443,7 @@ private class TestCallbackHandler
extends AMRMClientAsync.AbstractCallbackHandler {
private volatile List<ContainerStatus> completedContainers;
private volatile List<Container> allocatedContainers;
private final List<Container> changedContainers = new ArrayList<>();
private final List<UpdatedContainer> changedContainers = new ArrayList<>();
Exception savedException = null;
volatile boolean reboot = false;
Object notifier = new Object();
@ -448,8 +462,8 @@ public List<ContainerStatus> takeCompletedContainers() {
return ret;
}
public List<Container> takeChangedContainers() {
List<Container> ret = null;
public List<UpdatedContainer> takeChangedContainers() {
List<UpdatedContainer> ret = null;
synchronized (changedContainers) {
if (!changedContainers.isEmpty()) {
ret = new ArrayList<>(changedContainers);
@ -488,8 +502,8 @@ public void onContainersCompleted(List<ContainerStatus> statuses) {
}
@Override
public void onContainersResourceChanged(
List<Container> changed) {
public void onContainersUpdated(
List<UpdatedContainer> changed) {
synchronized (changedContainers) {
changedContainers.clear();
changedContainers.addAll(changed);
@ -564,7 +578,8 @@ public void onContainersCompleted(List<ContainerStatus> statuses) {
public void onContainersAllocated(List<Container> containers) {}
@Override
public void onContainersResourceChanged(List<Container> containers) {}
public void onContainersUpdated(
List<UpdatedContainer> containers) {}
@Override
public void onShutdownRequest() {}

View File

@ -72,6 +72,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.ClientRMProxy;
import org.apache.hadoop.yarn.client.api.AMRMClient;
@ -1061,33 +1062,20 @@ private void doContainerResourceChange(
Assert.assertEquals(2, amClientImpl.pendingChange.size());
// as of now: container1 asks to decrease to (512, 1)
// container2 asks to increase to (2048, 1)
List<Container> decreasedContainers;
List<Container> increasedContainers;
int allocateAttempts = 0;
int decreased = 0;
int increased = 0;
while (allocateAttempts < 30) {
// send allocation requests
AllocateResponse allocResponse = amClient.allocate(0.1f);
decreasedContainers = allocResponse.getDecreasedContainers();
increasedContainers = allocResponse.getIncreasedContainers();
decreased += decreasedContainers.size();
increased += increasedContainers.size();
if (allocateAttempts == 0) {
// we should get decrease confirmation right away
Assert.assertEquals(1, decreased);
// After first allocate request check change size
Assert.assertEquals(0, amClientImpl.change.size());
} else if (increased == 1) {
break;
}
// increase request is served after next NM heart beat is received
// Sleeping and retrying allocate
sleep(20);
allocateAttempts++;
}
Assert.assertEquals(1, decreased);
Assert.assertEquals(1, increased);
// send allocation requests
AllocateResponse allocResponse = amClient.allocate(0.1f);
Assert.assertEquals(0, amClientImpl.change.size());
// we should get decrease confirmation right away
List<UpdatedContainer> updatedContainers =
allocResponse.getUpdatedContainers();
Assert.assertEquals(1, updatedContainers.size());
// we should get increase allocation after the next NM's heartbeat to RM
sleep(150);
// get allocations
allocResponse = amClient.allocate(0.1f);
updatedContainers =
allocResponse.getUpdatedContainers();
Assert.assertEquals(1, updatedContainers.size());
}
private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)

View File

@ -39,12 +39,12 @@
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -262,7 +262,7 @@ public void testAMRMClientResendsRequestsOnRMRestart() throws Exception {
// new NM to represent NM re-register
nm1 = new MockNM("h1:1234", 10240, rm2.getResourceTrackerService());
NMContainerStatus containerReport =
NMContainerStatus.newInstance(containerId, ContainerState.RUNNING,
NMContainerStatus.newInstance(containerId, 0, ContainerState.RUNNING,
Resource.newInstance(1024, 1), "recover container", 0,
Priority.newInstance(0), 0);
nm1.registerNode(Collections.singletonList(containerReport),
@ -399,7 +399,7 @@ public void testAMRMClientForUnregisterAMOnRMRestart() throws Exception {
ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
NMContainerStatus containerReport =
NMContainerStatus.newInstance(containerId, ContainerState.RUNNING,
NMContainerStatus.newInstance(containerId, 0, ContainerState.RUNNING,
Resource.newInstance(1024, 1), "recover container", 0,
Priority.newInstance(0), 0);
nm1.registerNode(Arrays.asList(containerReport), null);
@ -562,8 +562,8 @@ public MyFifoScheduler(RMContext rmContext) {
List<ResourceRequest> lastAsk = null;
List<ContainerId> lastRelease = null;
List<ContainerResourceChangeRequest> lastIncrease = null;
List<ContainerResourceChangeRequest> lastDecrease = null;
List<UpdateContainerRequest> lastIncrease = null;
List<UpdateContainerRequest> lastDecrease = null;
List<String> lastBlacklistAdditions;
List<String> lastBlacklistRemovals;
@ -574,8 +574,8 @@ public synchronized Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests) {
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
for (ResourceRequest req : ask) {
ResourceRequest reqCopy =

View File

@ -27,17 +27,17 @@
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.UpdateContainerRequestProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProtoOrBuilder;
@ -52,8 +52,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
private List<ResourceRequest> ask = null;
private List<ContainerId> release = null;
private List<ContainerResourceChangeRequest> increaseRequests = null;
private List<ContainerResourceChangeRequest> decreaseRequests = null;
private List<UpdateContainerRequest> updateRequests = null;
private ResourceBlacklistRequest blacklistRequest = null;
public AllocateRequestPBImpl() {
@ -99,11 +98,8 @@ private void mergeLocalToBuilder() {
if (this.release != null) {
addReleasesToProto();
}
if (this.increaseRequests != null) {
addIncreaseRequestsToProto();
}
if (this.decreaseRequests != null) {
addDecreaseRequestsToProto();
if (this.updateRequests != null) {
addUpdateRequestsToProto();
}
if (this.blacklistRequest != null) {
builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
@ -166,37 +162,19 @@ public void setAskList(final List<ResourceRequest> resourceRequests) {
}
@Override
public List<ContainerResourceChangeRequest> getIncreaseRequests() {
initIncreaseRequests();
return this.increaseRequests;
public List<UpdateContainerRequest> getUpdateRequests() {
initUpdateRequests();
return this.updateRequests;
}
@Override
public void setIncreaseRequests(
List<ContainerResourceChangeRequest> increaseRequests) {
if (increaseRequests == null) {
public void setUpdateRequests(List<UpdateContainerRequest> updateRequests) {
if (updateRequests == null) {
return;
}
initIncreaseRequests();
this.increaseRequests.clear();
this.increaseRequests.addAll(increaseRequests);
}
@Override
public List<ContainerResourceChangeRequest> getDecreaseRequests() {
initDecreaseRequests();
return this.decreaseRequests;
}
@Override
public void setDecreaseRequests(
List<ContainerResourceChangeRequest> decreaseRequests) {
if (decreaseRequests == null) {
return;
}
initDecreaseRequests();
this.decreaseRequests.clear();
this.decreaseRequests.addAll(decreaseRequests);
initUpdateRequests();
this.updateRequests.clear();
this.updateRequests.addAll(updateRequests);
}
@Override
@ -239,7 +217,8 @@ private void addAsksToProto() {
builder.clearAsk();
if (ask == null)
return;
Iterable<ResourceRequestProto> iterable = new Iterable<ResourceRequestProto>() {
Iterable<ResourceRequestProto> iterable =
new Iterable<ResourceRequestProto>() {
@Override
public Iterator<ResourceRequestProto> iterator() {
return new Iterator<ResourceRequestProto>() {
@ -268,48 +247,34 @@ public void remove() {
builder.addAllAsk(iterable);
}
private void initIncreaseRequests() {
if (this.increaseRequests != null) {
private void initUpdateRequests() {
if (this.updateRequests != null) {
return;
}
AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
List<ContainerResourceChangeRequestProto> list =
p.getIncreaseRequestList();
this.increaseRequests = new ArrayList<ContainerResourceChangeRequest>();
List<UpdateContainerRequestProto> list =
p.getUpdateRequestsList();
this.updateRequests = new ArrayList<>();
for (ContainerResourceChangeRequestProto c : list) {
this.increaseRequests.add(convertFromProtoFormat(c));
for (UpdateContainerRequestProto c : list) {
this.updateRequests.add(convertFromProtoFormat(c));
}
}
private void initDecreaseRequests() {
if (this.decreaseRequests != null) {
return;
}
AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
List<ContainerResourceChangeRequestProto> list =
p.getDecreaseRequestList();
this.decreaseRequests = new ArrayList<>();
for (ContainerResourceChangeRequestProto c : list) {
this.decreaseRequests.add(convertFromProtoFormat(c));
}
}
private void addIncreaseRequestsToProto() {
private void addUpdateRequestsToProto() {
maybeInitBuilder();
builder.clearIncreaseRequest();
if (increaseRequests == null) {
builder.clearUpdateRequests();
if (updateRequests == null) {
return;
}
Iterable<ContainerResourceChangeRequestProto> iterable =
new Iterable<ContainerResourceChangeRequestProto>() {
Iterable<UpdateContainerRequestProto> iterable =
new Iterable<UpdateContainerRequestProto>() {
@Override
public Iterator<ContainerResourceChangeRequestProto> iterator() {
return new Iterator<ContainerResourceChangeRequestProto>() {
public Iterator<UpdateContainerRequestProto> iterator() {
return new Iterator<UpdateContainerRequestProto>() {
Iterator<ContainerResourceChangeRequest> iter =
increaseRequests.iterator();
private Iterator<UpdateContainerRequest> iter =
updateRequests.iterator();
@Override
public boolean hasNext() {
@ -317,7 +282,7 @@ public boolean hasNext() {
}
@Override
public ContainerResourceChangeRequestProto next() {
public UpdateContainerRequestProto next() {
return convertToProtoFormat(iter.next());
}
@ -329,43 +294,7 @@ public void remove() {
}
};
builder.addAllIncreaseRequest(iterable);
}
private void addDecreaseRequestsToProto() {
maybeInitBuilder();
builder.clearDecreaseRequest();
if (decreaseRequests == null) {
return;
}
Iterable<ContainerResourceChangeRequestProto> iterable =
new Iterable<ContainerResourceChangeRequestProto>() {
@Override
public Iterator<ContainerResourceChangeRequestProto> iterator() {
return new Iterator<ContainerResourceChangeRequestProto>() {
Iterator<ContainerResourceChangeRequest> iter =
decreaseRequests.iterator();
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public ContainerResourceChangeRequestProto next() {
return convertToProtoFormat(iter.next());
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
};
builder.addAllDecreaseRequest(iterable);
builder.addAllUpdateRequests(iterable);
}
@Override
@ -438,14 +367,14 @@ private ResourceRequestProto convertToProtoFormat(ResourceRequest t) {
return ((ResourceRequestPBImpl)t).getProto();
}
private ContainerResourceChangeRequestPBImpl convertFromProtoFormat(
ContainerResourceChangeRequestProto p) {
return new ContainerResourceChangeRequestPBImpl(p);
private UpdateContainerRequestPBImpl convertFromProtoFormat(
UpdateContainerRequestProto p) {
return new UpdateContainerRequestPBImpl(p);
}
private ContainerResourceChangeRequestProto convertToProtoFormat(
ContainerResourceChangeRequest t) {
return ((ContainerResourceChangeRequestPBImpl) t).getProto();
private UpdateContainerRequestProto convertToProtoFormat(
UpdateContainerRequest t) {
return ((UpdateContainerRequestPBImpl) t).getProto();
}
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {

View File

@ -36,6 +36,8 @@
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
@ -45,12 +47,14 @@
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.UpdatedContainerPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.NMTokenProto;
@ -69,10 +73,10 @@ public class AllocateResponsePBImpl extends AllocateResponse {
private List<Container> allocatedContainers = null;
private List<NMToken> nmTokens = null;
private List<ContainerStatus> completedContainersStatuses = null;
private List<Container> increasedContainers = null;
private List<Container> decreasedContainers = null;
private List<UpdatedContainer> updatedContainers = null;
private List<NodeReport> updatedNodes = null;
private List<UpdateContainerError> updateErrors = null;
private PreemptionMessage preempt;
private Token amrmToken = null;
private Priority appPriority = null;
@ -143,17 +147,17 @@ private synchronized void mergeLocalToBuilder() {
if (this.preempt != null) {
builder.setPreempt(convertToProtoFormat(this.preempt));
}
if (this.increasedContainers != null) {
builder.clearIncreasedContainers();
Iterable<ContainerProto> iterable =
getContainerProtoIterable(this.increasedContainers);
builder.addAllIncreasedContainers(iterable);
if (this.updatedContainers != null) {
builder.clearUpdatedContainers();
Iterable<YarnServiceProtos.UpdatedContainerProto> iterable =
getUpdatedContainerProtoIterable(this.updatedContainers);
builder.addAllUpdatedContainers(iterable);
}
if (this.decreasedContainers != null) {
builder.clearDecreasedContainers();
Iterable<ContainerProto> iterable =
getContainerProtoIterable(this.decreasedContainers);
builder.addAllDecreasedContainers(iterable);
if (this.updateErrors != null) {
builder.clearUpdateErrors();
Iterable<YarnServiceProtos.UpdateContainerErrorProto> iterable =
getUpdateErrorsIterable(this.updateErrors);
builder.addAllUpdateErrors(iterable);
}
if (this.amrmToken != null) {
builder.setAmRmToken(convertToProtoFormat(this.amrmToken));
@ -247,6 +251,24 @@ public synchronized void setUpdatedNodes(
this.updatedNodes.addAll(updatedNodes);
}
@Override
public synchronized List<UpdateContainerError> getUpdateErrors() {
initLocalUpdateErrorsList();
return this.updateErrors;
}
@Override
public synchronized void setUpdateErrors(
List<UpdateContainerError> updateErrors) {
if (updateErrors == null) {
this.updateErrors.clear();
return;
}
this.updateErrors = new ArrayList<>(
updateErrors.size());
this.updateErrors.addAll(updateErrors);
}
@Override
public synchronized List<Container> getAllocatedContainers() {
initLocalNewContainerList();
@ -264,33 +286,18 @@ public synchronized void setAllocatedContainers(
}
@Override
public synchronized List<Container> getIncreasedContainers() {
initLocalIncreasedContainerList();
return this.increasedContainers;
public synchronized List<UpdatedContainer> getUpdatedContainers() {
initLocalUpdatedContainerList();
return this.updatedContainers;
}
@Override
public synchronized void setIncreasedContainers(
final List<Container> containers) {
public synchronized void setUpdatedContainers(
final List<UpdatedContainer> containers) {
if (containers == null)
return;
initLocalIncreasedContainerList();
increasedContainers.addAll(containers);
}
@Override
public synchronized List<Container> getDecreasedContainers() {
initLocalDecreasedContainerList();
return this.decreasedContainers;
}
@Override
public synchronized void setDecreasedContainers(
final List<Container> containers) {
if (containers == null)
return;
initLocalDecreasedContainerList();
decreasedContainers.addAll(containers);
initLocalUpdatedContainerList();
updatedContainers.addAll(containers);
}
//// Finished containers
@ -406,29 +413,17 @@ public synchronized void setApplicationPriority(Priority priority) {
this.appPriority = priority;
}
private synchronized void initLocalIncreasedContainerList() {
if (this.increasedContainers != null) {
private synchronized void initLocalUpdatedContainerList() {
if (this.updatedContainers != null) {
return;
}
AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
List<ContainerProto> list = p.getIncreasedContainersList();
increasedContainers = new ArrayList<>();
List<YarnServiceProtos.UpdatedContainerProto> list =
p.getUpdatedContainersList();
updatedContainers = new ArrayList<>();
for (ContainerProto c : list) {
increasedContainers.add(convertFromProtoFormat(c));
}
}
private synchronized void initLocalDecreasedContainerList() {
if (this.decreasedContainers != null) {
return;
}
AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
List<ContainerProto> list = p.getDecreasedContainersList();
decreasedContainers = new ArrayList<>();
for (ContainerProto c : list) {
decreasedContainers.add(convertFromProtoFormat(c));
for (YarnServiceProtos.UpdatedContainerProto c : list) {
updatedContainers.add(convertFromProtoFormat(c));
}
}
@ -474,6 +469,53 @@ private synchronized void initLocalNewNMTokenList() {
}
}
private synchronized void initLocalUpdateErrorsList() {
if (updateErrors != null) {
return;
}
AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
List<YarnServiceProtos.UpdateContainerErrorProto> list =
p.getUpdateErrorsList();
this.updateErrors = new ArrayList<UpdateContainerError>();
for (YarnServiceProtos.UpdateContainerErrorProto t : list) {
updateErrors.add(ProtoUtils.convertFromProtoFormat(t));
}
}
private synchronized Iterable<YarnServiceProtos.UpdateContainerErrorProto>
getUpdateErrorsIterable(
final List<UpdateContainerError> updateErrorsList) {
maybeInitBuilder();
return new Iterable<YarnServiceProtos.UpdateContainerErrorProto>() {
@Override
public synchronized Iterator<YarnServiceProtos
.UpdateContainerErrorProto> iterator() {
return new Iterator<YarnServiceProtos.UpdateContainerErrorProto>() {
private Iterator<UpdateContainerError> iter =
updateErrorsList.iterator();
@Override
public synchronized boolean hasNext() {
return iter.hasNext();
}
@Override
public synchronized YarnServiceProtos.UpdateContainerErrorProto
next() {
return ProtoUtils.convertToProtoFormat(iter.next());
}
@Override
public synchronized void remove() {
throw new UnsupportedOperationException();
}
};
}
};
}
private synchronized Iterable<ContainerProto> getContainerProtoIterable(
final List<Container> newContainersList) {
maybeInitBuilder();
@ -505,6 +547,40 @@ public synchronized void remove() {
};
}
private synchronized Iterable<YarnServiceProtos.UpdatedContainerProto>
getUpdatedContainerProtoIterable(
final List<UpdatedContainer> newUpdatedContainersList) {
maybeInitBuilder();
return new Iterable<YarnServiceProtos.UpdatedContainerProto>() {
@Override
public synchronized Iterator<YarnServiceProtos.UpdatedContainerProto>
iterator() {
return new Iterator<YarnServiceProtos.UpdatedContainerProto>() {
private Iterator<UpdatedContainer> iter =
newUpdatedContainersList.iterator();
@Override
public synchronized boolean hasNext() {
return iter.hasNext();
}
@Override
public synchronized YarnServiceProtos.UpdatedContainerProto next() {
return convertToProtoFormat(iter.next());
}
@Override
public synchronized void remove() {
throw new UnsupportedOperationException();
}
};
}
};
}
private synchronized Iterable<NMTokenProto> getTokenProtoIterable(
final List<NMToken> nmTokenList) {
maybeInitBuilder();
@ -631,6 +707,16 @@ private synchronized ContainerProto convertToProtoFormat(
return ((ContainerPBImpl)t).getProto();
}
private synchronized UpdatedContainerPBImpl convertFromProtoFormat(
YarnServiceProtos.UpdatedContainerProto p) {
return new UpdatedContainerPBImpl(p);
}
private synchronized YarnServiceProtos.UpdatedContainerProto
convertToProtoFormat(UpdatedContainer t) {
return ((UpdatedContainerPBImpl)t).getProto();
}
private synchronized ContainerStatusPBImpl convertFromProtoFormat(
ContainerStatusProto p) {
return new ContainerStatusPBImpl(p);

View File

@ -274,6 +274,18 @@ public void setAllocationRequestId(long allocationRequestID) {
builder.setAllocationRequestId(allocationRequestID);
}
@Override
public int getVersion() {
ContainerProtoOrBuilder p = viaProto ? proto : builder;
return p.getVersion();
}
@Override
public void setVersion(int version) {
maybeInitBuilder();
builder.setVersion(version);
}
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
return new ContainerIdPBImpl(p);
}
@ -329,6 +341,7 @@ public String toString() {
sb.append("ContainerId: ").append(getId()).append(", ");
sb.append("AllocationRequestId: ").append(getAllocationRequestId())
.append(", ");
sb.append("Version: ").append(getVersion()).append(", ");
sb.append("NodeId: ").append(getNodeId()).append(", ");
sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
sb.append("Resource: ").append(getResource()).append(", ");

View File

@ -1,141 +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.api.records.impl.pb;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
public class ContainerResourceChangeRequestPBImpl extends
ContainerResourceChangeRequest {
ContainerResourceChangeRequestProto proto =
ContainerResourceChangeRequestProto.getDefaultInstance();
ContainerResourceChangeRequestProto.Builder builder = null;
boolean viaProto = false;
private ContainerId existingContainerId = null;
private Resource targetCapability = null;
public ContainerResourceChangeRequestPBImpl() {
builder = ContainerResourceChangeRequestProto.newBuilder();
}
public ContainerResourceChangeRequestPBImpl(
ContainerResourceChangeRequestProto proto) {
this.proto = proto;
viaProto = true;
}
public ContainerResourceChangeRequestProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
@Override
public ContainerId getContainerId() {
ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto
: builder;
if (this.existingContainerId != null) {
return this.existingContainerId;
}
if (p.hasContainerId()) {
this.existingContainerId = convertFromProtoFormat(p.getContainerId());
}
return this.existingContainerId;
}
@Override
public void setContainerId(ContainerId existingContainerId) {
maybeInitBuilder();
if (existingContainerId == null) {
builder.clearContainerId();
}
this.existingContainerId = existingContainerId;
}
@Override
public Resource getCapability() {
ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto
: builder;
if (this.targetCapability != null) {
return this.targetCapability;
}
if (p.hasCapability()) {
this.targetCapability = convertFromProtoFormat(p.getCapability());
}
return this.targetCapability;
}
@Override
public void setCapability(Resource targetCapability) {
maybeInitBuilder();
if (targetCapability == null) {
builder.clearCapability();
}
this.targetCapability = targetCapability;
}
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
return new ContainerIdPBImpl(p);
}
private ContainerIdProto convertToProtoFormat(ContainerId t) {
return ((ContainerIdPBImpl) t).getProto();
}
private Resource convertFromProtoFormat(ResourceProto p) {
return new ResourcePBImpl(p);
}
private ResourceProto convertToProtoFormat(Resource t) {
return ((ResourcePBImpl) t).getProto();
}
private void mergeLocalToProto() {
if (viaProto) {
maybeInitBuilder();
}
mergeLocalToBuilder();
proto = builder.build();
viaProto = true;
}
private void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = ContainerResourceChangeRequestProto.newBuilder(proto);
}
viaProto = false;
}
private void mergeLocalToBuilder() {
if (this.existingContainerId != null) {
builder.setContainerId(convertToProtoFormat(this.existingContainerId));
}
if (this.targetCapability != null) {
builder.setCapability(convertToProtoFormat(this.targetCapability));
}
}
}

View File

@ -27,8 +27,10 @@
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@ -41,12 +43,15 @@
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.proto.YarnProtos;
import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceTypeProto;
@ -57,6 +62,7 @@
import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryPolicyProto;
@ -64,6 +70,7 @@
import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateTypeProto;
import org.apache.hadoop.yarn.server.api.ContainerType;
import com.google.protobuf.ByteString;
@ -315,16 +322,26 @@ public static ExecutionType convertFromProtoFormat(ExecutionTypeProto e) {
return ExecutionType.valueOf(e.name());
}
/*
* ContainerUpdateType
*/
public static ContainerUpdateTypeProto convertToProtoFormat(
ContainerUpdateType e) {
return ContainerUpdateTypeProto.valueOf(e.name());
}
public static ContainerUpdateType convertFromProtoFormat(
ContainerUpdateTypeProto e) {
return ContainerUpdateType.valueOf(e.name());
}
/*
* Resource
*/
public static synchronized YarnProtos.ResourceProto convertToProtoFormat(
Resource r) {
public static synchronized ResourceProto convertToProtoFormat(Resource r) {
return ((ResourcePBImpl) r).getProto();
}
public static Resource convertFromProtoFormat(
YarnProtos.ResourceProto resource) {
public static Resource convertFromProtoFormat(ResourceProto resource) {
return new ResourcePBImpl(resource);
}
@ -349,8 +366,52 @@ public static YarnProtos.ContainerProto convertToProtoFormat(
return ((ContainerPBImpl)t).getProto();
}
public static ContainerPBImpl convertFromProtoFormat(
YarnProtos.ContainerProto t) {
return new ContainerPBImpl(t);
}
public static ContainerStatusPBImpl convertFromProtoFormat(
YarnProtos.ContainerStatusProto p) {
return new ContainerStatusPBImpl(p);
}
/*
* ContainerId
*/
public static ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
return new ContainerIdPBImpl(p);
}
public static ContainerIdProto convertToProtoFormat(ContainerId t) {
return ((ContainerIdPBImpl) t).getProto();
}
/*
* UpdateContainerRequest
*/
public static UpdateContainerRequestPBImpl convertFromProtoFormat(
YarnServiceProtos.UpdateContainerRequestProto p) {
return new UpdateContainerRequestPBImpl(p);
}
public static YarnServiceProtos.UpdateContainerRequestProto
convertToProtoFormat(UpdateContainerRequest t) {
return ((UpdateContainerRequestPBImpl) t).getProto();
}
/*
* UpdateContainerError
*/
public static UpdateContainerErrorPBImpl convertFromProtoFormat(
YarnServiceProtos.UpdateContainerErrorProto p) {
return new UpdateContainerErrorPBImpl(p);
}
public static YarnServiceProtos.UpdateContainerErrorProto
convertToProtoFormat(UpdateContainerError t) {
return ((UpdateContainerErrorPBImpl) t).getProto();
}
}

View File

@ -0,0 +1,125 @@
/**
* 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.api.records.impl.pb;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.proto.YarnServiceProtos;
/**
* Implementation of <code>UpdateContainerError</code>.
*/
public class UpdateContainerErrorPBImpl extends UpdateContainerError {
private YarnServiceProtos.UpdateContainerErrorProto proto =
YarnServiceProtos.UpdateContainerErrorProto.getDefaultInstance();
private YarnServiceProtos.UpdateContainerErrorProto.Builder builder = null;
private boolean viaProto = false;
private String reason = null;
private UpdateContainerRequest updateRequest = null;
public UpdateContainerErrorPBImpl() {
builder = YarnServiceProtos.UpdateContainerErrorProto.newBuilder();
}
public UpdateContainerErrorPBImpl(YarnServiceProtos
.UpdateContainerErrorProto proto) {
this.proto = proto;
viaProto = true;
}
public YarnServiceProtos.UpdateContainerErrorProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
@Override
public String getReason() {
YarnServiceProtos.UpdateContainerErrorProtoOrBuilder p = viaProto ? proto
: builder;
if (this.reason != null) {
return this.reason;
}
if (p.hasReason()) {
this.reason = p.getReason();
}
return this.reason;
}
@Override
public void setReason(String reason) {
maybeInitBuilder();
if (reason == null) {
builder.clearReason();
}
this.reason = reason;
}
@Override
public UpdateContainerRequest getUpdateContainerRequest() {
YarnServiceProtos.UpdateContainerErrorProtoOrBuilder p = viaProto ? proto
: builder;
if (this.updateRequest != null) {
return this.updateRequest;
}
if (p.hasUpdateRequest()) {
this.updateRequest =
ProtoUtils.convertFromProtoFormat(p.getUpdateRequest());
}
return this.updateRequest;
}
@Override
public void setUpdateContainerRequest(
UpdateContainerRequest updateContainerRequest) {
maybeInitBuilder();
if (updateContainerRequest == null) {
builder.clearUpdateRequest();
}
this.updateRequest = updateContainerRequest;
}
private void mergeLocalToProto() {
if (viaProto) {
maybeInitBuilder();
}
mergeLocalToBuilder();
proto = builder.build();
viaProto = true;
}
private void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = YarnServiceProtos.UpdateContainerErrorProto.newBuilder(proto);
}
viaProto = false;
}
private void mergeLocalToBuilder() {
if (this.reason != null) {
builder.setReason(this.reason);
}
if (this.updateRequest != null) {
builder.setUpdateRequest(
ProtoUtils.convertToProtoFormat(this.updateRequest));
}
}
}

View File

@ -0,0 +1,187 @@
/**
* 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.api.records.impl.pb;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.proto.YarnServiceProtos;
/**
* Implementation of <code>UpdateContainerRequest</code>.
*/
public class UpdateContainerRequestPBImpl extends UpdateContainerRequest {
private YarnServiceProtos.UpdateContainerRequestProto proto =
YarnServiceProtos.UpdateContainerRequestProto.getDefaultInstance();
private YarnServiceProtos.UpdateContainerRequestProto.Builder builder = null;
private boolean viaProto = false;
private ContainerId existingContainerId = null;
private Resource targetCapability = null;
public UpdateContainerRequestPBImpl() {
builder = YarnServiceProtos.UpdateContainerRequestProto.newBuilder();
}
public UpdateContainerRequestPBImpl(YarnServiceProtos
.UpdateContainerRequestProto proto) {
this.proto = proto;
viaProto = true;
}
public YarnServiceProtos.UpdateContainerRequestProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
@Override
public int getContainerVersion() {
YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
viaProto ? proto : builder;
if (!p.hasContainerVersion()) {
return 0;
}
return p.getContainerVersion();
}
@Override
public void setContainerVersion(int containerVersion) {
maybeInitBuilder();
builder.setContainerVersion(containerVersion);
}
@Override
public ContainerId getContainerId() {
YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
viaProto ? proto : builder;
if (this.existingContainerId != null) {
return this.existingContainerId;
}
if (p.hasContainerId()) {
this.existingContainerId =
ProtoUtils.convertFromProtoFormat(p.getContainerId());
}
return this.existingContainerId;
}
@Override
public void setContainerId(ContainerId containerId) {
maybeInitBuilder();
if (containerId == null) {
builder.clearContainerId();
}
this.existingContainerId = containerId;
}
@Override
public Resource getCapability() {
YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p = viaProto ? proto
: builder;
if (this.targetCapability != null) {
return this.targetCapability;
}
if (p.hasCapability()) {
this.targetCapability =
ProtoUtils.convertFromProtoFormat(p.getCapability());
}
return this.targetCapability;
}
@Override
public void setCapability(Resource capability) {
maybeInitBuilder();
if (capability == null) {
builder.clearCapability();
}
this.targetCapability = capability;
}
@Override
public ExecutionType getExecutionType() {
YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
viaProto ? proto : builder;
if (!p.hasExecutionType()) {
return null;
}
return ProtoUtils.convertFromProtoFormat(p.getExecutionType());
}
@Override
public void setExecutionType(ExecutionType execType) {
maybeInitBuilder();
if (execType == null) {
builder.clearExecutionType();
return;
}
builder.setExecutionType(ProtoUtils.convertToProtoFormat(execType));
}
@Override
public ContainerUpdateType getContainerUpdateType() {
YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
viaProto ? proto : builder;
if (!p.hasUpdateType()) {
return null;
}
return ProtoUtils.convertFromProtoFormat(p.getUpdateType());
}
@Override
public void setContainerUpdateType(ContainerUpdateType updateType) {
maybeInitBuilder();
if (updateType == null) {
builder.clearUpdateType();
return;
}
builder.setUpdateType(ProtoUtils.convertToProtoFormat(updateType));
}
private void mergeLocalToProto() {
if (viaProto) {
maybeInitBuilder();
}
mergeLocalToBuilder();
proto = builder.build();
viaProto = true;
}
private void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = YarnServiceProtos.UpdateContainerRequestProto.newBuilder(proto);
}
viaProto = false;
}
private void mergeLocalToBuilder() {
if (this.existingContainerId != null) {
builder.setContainerId(
ProtoUtils.convertToProtoFormat(this.existingContainerId));
}
if (this.targetCapability != null) {
builder.setCapability(
ProtoUtils.convertToProtoFormat(this.targetCapability));
}
}
}

View File

@ -0,0 +1,117 @@
/**
* 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.api.records.impl.pb;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.proto.YarnServiceProtos;
/**
* Implementation of <code>UpdatedContainer</code>.
*/
public class UpdatedContainerPBImpl extends UpdatedContainer {
private YarnServiceProtos.UpdatedContainerProto proto =
YarnServiceProtos.UpdatedContainerProto.getDefaultInstance();
private YarnServiceProtos.UpdatedContainerProto.Builder builder = null;
private boolean viaProto = false;
private Container container = null;
public UpdatedContainerPBImpl() {
builder = YarnServiceProtos.UpdatedContainerProto.newBuilder();
}
public UpdatedContainerPBImpl(YarnServiceProtos.UpdatedContainerProto proto) {
this.proto = proto;
viaProto = true;
}
private void mergeLocalToBuilder() {
if (this.container != null) {
builder.setContainer(ProtoUtils.convertToProtoFormat(this.container));
}
}
private void mergeLocalToProto() {
if (viaProto) {
maybeInitBuilder();
}
mergeLocalToBuilder();
proto = builder.build();
viaProto = true;
}
private void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = YarnServiceProtos.UpdatedContainerProto.newBuilder(proto);
}
viaProto = false;
}
public YarnServiceProtos.UpdatedContainerProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
@Override
public ContainerUpdateType getUpdateType() {
YarnServiceProtos.UpdatedContainerProtoOrBuilder p =
viaProto ? proto : builder;
if (!p.hasUpdateType()) {
return null;
}
return ProtoUtils.convertFromProtoFormat(p.getUpdateType());
}
@Override
public void setUpdateType(ContainerUpdateType updateType) {
maybeInitBuilder();
if (updateType == null) {
builder.clearUpdateType();
return;
}
builder.setUpdateType(ProtoUtils.convertToProtoFormat(updateType));
}
@Override
public Container getContainer() {
YarnServiceProtos.UpdatedContainerProtoOrBuilder p =
viaProto ? proto : builder;
if (this.container != null) {
return this.container;
}
if (!p.hasContainer()) {
return null;
}
this.container = ProtoUtils.convertFromProtoFormat(p.getContainer());
return this.container;
}
@Override
public void setContainer(Container container) {
maybeInitBuilder();
if (container == null) {
builder.clearContainer();
}
this.container = container;
}
}

View File

@ -87,14 +87,15 @@ public ContainerTokenIdentifier(ContainerId containerID, String hostName,
long rmIdentifier, Priority priority, long creationTime,
LogAggregationContext logAggregationContext, String nodeLabelExpression,
ContainerType containerType) {
this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
rmIdentifier, priority, creationTime, logAggregationContext,
nodeLabelExpression, containerType, ExecutionType.GUARANTEED);
this(containerID, 0, hostName, appSubmitter, r, expiryTimeStamp,
masterKeyId, rmIdentifier, priority, creationTime,
logAggregationContext, nodeLabelExpression, containerType,
ExecutionType.GUARANTEED);
}
public ContainerTokenIdentifier(ContainerId containerID, String hostName,
String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
long rmIdentifier, Priority priority, long creationTime,
public ContainerTokenIdentifier(ContainerId containerID, int containerVersion,
String hostName, String appSubmitter, Resource r, long expiryTimeStamp,
int masterKeyId, long rmIdentifier, Priority priority, long creationTime,
LogAggregationContext logAggregationContext, String nodeLabelExpression,
ContainerType containerType, ExecutionType executionType) {
ContainerTokenIdentifierProto.Builder builder =
@ -102,6 +103,7 @@ public ContainerTokenIdentifier(ContainerId containerID, String hostName,
if (containerID != null) {
builder.setContainerId(((ContainerIdPBImpl)containerID).getProto());
}
builder.setVersion(containerVersion);
builder.setNmHostAddr(hostName);
builder.setAppSubmitter(appSubmitter);
if (r != null) {
@ -184,7 +186,7 @@ public long getRMIdentifier() {
}
/**
* Get the ContainerType of container to allocate
* Get the ContainerType of container to allocate.
* @return ContainerType
*/
public ContainerType getContainerType(){
@ -241,7 +243,18 @@ public UserGroupInformation getUser() {
return UserGroupInformation.createRemoteUser(
containerId);
}
/**
* Get the Container version
* @return container version
*/
public int getVersion() {
if (proto.hasVersion()) {
return proto.getVersion();
} else {
return 0;
}
}
/**
* Get the node-label-expression in the original ResourceRequest
*/

View File

@ -52,6 +52,7 @@ message ContainerTokenIdentifierProto {
optional string nodeLabelExpression = 11;
optional ContainerTypeProto containerType = 12;
optional ExecutionTypeProto executionType = 13 [default = GUARANTEED];
optional int32 version = 14 [default = 0];
}
message ClientToAMTokenIdentifierProto {

View File

@ -121,7 +121,6 @@
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerReport;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
@ -155,6 +154,9 @@
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl;
@ -166,7 +168,6 @@
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerRetryContextPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ExecutionTypeRequestPBImpl;
@ -190,6 +191,7 @@
import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.YarnClusterMetricsPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto;
@ -201,7 +203,6 @@
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryContextProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
@ -489,7 +490,8 @@ public static void setup() throws Exception {
generateByNewInstance(ContainerLaunchContext.class);
generateByNewInstance(ApplicationSubmissionContext.class);
generateByNewInstance(ContainerReport.class);
generateByNewInstance(ContainerResourceChangeRequest.class);
generateByNewInstance(UpdateContainerRequest.class);
generateByNewInstance(UpdateContainerError.class);
generateByNewInstance(IncreaseContainersResourceRequest.class);
generateByNewInstance(IncreaseContainersResourceResponse.class);
generateByNewInstance(ContainerStatus.class);
@ -501,6 +503,7 @@ public static void setup() throws Exception {
generateByNewInstance(PreemptionMessage.class);
generateByNewInstance(StartContainerRequest.class);
generateByNewInstance(NodeLabel.class);
generateByNewInstance(UpdatedContainer.class);
// genByNewInstance does not apply to QueueInfo, cause
// it is recursive(has sub queues)
typeValueCache.put(QueueInfo.class, QueueInfo.newInstance("root", 1.0f,
@ -1010,9 +1013,9 @@ public void testContainerReportPBImpl() throws Exception {
}
@Test
public void testContainerResourceChangeRequestPBImpl() throws Exception {
validatePBImplRecord(ContainerResourceChangeRequestPBImpl.class,
ContainerResourceChangeRequestProto.class);
public void testUpdateContainerRequestPBImpl() throws Exception {
validatePBImplRecord(UpdateContainerRequestPBImpl.class,
YarnServiceProtos.UpdateContainerRequestProto.class);
}
@Test

View File

@ -153,7 +153,7 @@ public void testContainerTokenIdentifier() throws IOException {
long creationTime = 1000;
ContainerTokenIdentifier token = new ContainerTokenIdentifier(
containerID, hostName, appSubmitter, r, expiryTimeStamp,
containerID, hostName, appSubmitter, r, expiryTimeStamp,
masterKeyId, rmIdentifier, priority, creationTime);
ContainerTokenIdentifier anotherToken = new ContainerTokenIdentifier();
@ -392,7 +392,7 @@ public void testAMContainerTokenIdentifier() throws IOException {
anotherToken.getExecutionType());
token =
new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
new ContainerTokenIdentifier(containerID, 0, hostName, appSubmitter, r,
expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
ExecutionType.OPPORTUNISTIC);

View File

@ -35,21 +35,22 @@ public abstract class NMContainerStatus {
// Used by tests only
public static NMContainerStatus newInstance(ContainerId containerId,
ContainerState containerState, Resource allocatedResource,
int version, ContainerState containerState, Resource allocatedResource,
String diagnostics, int containerExitStatus, Priority priority,
long creationTime) {
return newInstance(containerId, containerState, allocatedResource,
return newInstance(containerId, version, containerState, allocatedResource,
diagnostics, containerExitStatus, priority, creationTime,
CommonNodeLabelsManager.NO_LABEL);
}
public static NMContainerStatus newInstance(ContainerId containerId,
ContainerState containerState, Resource allocatedResource,
int version, ContainerState containerState, Resource allocatedResource,
String diagnostics, int containerExitStatus, Priority priority,
long creationTime, String nodeLabelExpression) {
NMContainerStatus status =
Records.newRecord(NMContainerStatus.class);
status.setContainerId(containerId);
status.setVersion(version);
status.setContainerState(containerState);
status.setAllocatedResource(allocatedResource);
status.setDiagnostics(diagnostics);
@ -125,4 +126,12 @@ public static NMContainerStatus newInstance(ContainerId containerId,
public abstract void setNodeLabelExpression(
String nodeLabelExpression);
public int getVersion() {
return 0;
}
public void setVersion(int version) {
}
}

View File

@ -83,6 +83,7 @@ public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("[").append(getContainerId()).append(", ")
.append("CreateTime: ").append(getCreationTime()).append(", ")
.append("Version: ").append(getVersion()).append(", ")
.append("State: ").append(getContainerState()).append(", ")
.append("Capability: ").append(getAllocatedResource()).append(", ")
.append("Diagnostics: ").append(getDiagnostics()).append(", ")
@ -184,6 +185,18 @@ public void setContainerExitStatus(int containerExitStatus) {
builder.setContainerExitStatus(containerExitStatus);
}
@Override
public int getVersion() {
NMContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
return p.getVersion();
}
@Override
public void setVersion(int version) {
maybeInitBuilder();
builder.setVersion(version);
}
@Override
public Priority getPriority() {
NMContainerStatusProtoOrBuilder p = viaProto ? proto : builder;

View File

@ -324,7 +324,7 @@ private Container buildContainer(long rmIdentifier,
long currTime = System.currentTimeMillis();
ContainerTokenIdentifier containerTokenIdentifier =
new ContainerTokenIdentifier(
cId, nodeId.getHost() + ":" + nodeId.getPort(), userName,
cId, 0, nodeId.getHost() + ":" + nodeId.getPort(), userName,
capability, currTime + appParams.containerTokenExpiryInterval,
tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
rr.getPriority(), currTime,

View File

@ -64,9 +64,11 @@
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.yarn.server.api.ContainerType;
/**
* Builder utilities to construct various objects.
@ -156,12 +158,14 @@ public static ContainerId newContainerId(int appId, int appAttemptId,
return cId;
}
public static Token newContainerToken(ContainerId cId, String host,
int port, String user, Resource r, long expiryTime, int masterKeyId,
byte[] password, long rmIdentifier) throws IOException {
public static Token newContainerToken(ContainerId cId, int containerVersion,
String host, int port, String user, Resource r, long expiryTime,
int masterKeyId, byte[] password, long rmIdentifier) throws IOException {
ContainerTokenIdentifier identifier =
new ContainerTokenIdentifier(cId, host + ":" + port, user, r,
expiryTime, masterKeyId, rmIdentifier, Priority.newInstance(0), 0);
new ContainerTokenIdentifier(cId, containerVersion, host + ":" + port,
user, r, expiryTime, masterKeyId, rmIdentifier,
Priority.newInstance(0), 0, null, CommonNodeLabelsManager.NO_LABEL,
ContainerType.TASK, ExecutionType.GUARANTEED);
return newContainerToken(BuilderUtils.newNodeId(host, port), password,
identifier);
}

View File

@ -129,6 +129,7 @@ message NMContainerStatusProto {
optional int32 container_exit_status = 6;
optional int64 creation_time = 7;
optional string nodeLabelExpression = 8;
optional int32 version = 9;
}
message SCMUploaderNotifyRequestProto {

View File

@ -63,7 +63,7 @@ public void testNMContainerStatus() {
Resource resource = Resource.newInstance(1000, 200);
NMContainerStatus report =
NMContainerStatus.newInstance(containerId,
NMContainerStatus.newInstance(containerId, 0,
ContainerState.COMPLETE, resource, "diagnostics",
ContainerExitStatus.ABORTED, Priority.newInstance(10), 1234);
NMContainerStatus reportProto =
@ -87,7 +87,7 @@ public void testRegisterNodeManagerRequest() {
ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
NMContainerStatus containerReport =
NMContainerStatus.newInstance(containerId,
NMContainerStatus.newInstance(containerId, 0,
ContainerState.RUNNING, Resource.newInstance(1024, 1), "diagnostics",
0, Priority.newInstance(10), 1234);
List<NMContainerStatus> reports = Arrays.asList(containerReport);

View File

@ -40,7 +40,7 @@ public void testRegisterNodeManagerRequest() {
"version", Arrays.asList(NMContainerStatus.newInstance(
ContainerId.newContainerId(
ApplicationAttemptId.newInstance(
ApplicationId.newInstance(1234L, 1), 1), 1),
ApplicationId.newInstance(1234L, 1), 1), 1), 0,
ContainerState.RUNNING, Resource.newInstance(1024, 1), "good", -1,
Priority.newInstance(0), 1234)), Arrays.asList(
ApplicationId.newInstance(1234L, 1),

View File

@ -963,7 +963,8 @@ protected void startContainerInternal(
logAggregationContext));
}
this.context.getNMStateStore().storeContainer(containerId, request);
this.context.getNMStateStore().storeContainer(containerId,
containerTokenIdentifier.getVersion(), request);
dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container));
@ -1047,7 +1048,8 @@ public IncreaseContainersResourceResponse increaseContainersResource(
// an updated NMToken.
updateNMTokenIdentifier(nmTokenIdentifier);
Resource resource = containerTokenIdentifier.getResource();
changeContainerResourceInternal(containerId, resource, true);
changeContainerResourceInternal(containerId,
containerTokenIdentifier.getVersion(), resource, true);
successfullyIncreasedContainers.add(containerId);
} catch (YarnException | InvalidToken e) {
failedContainers.put(containerId, SerializedException.newInstance(e));
@ -1061,8 +1063,8 @@ public IncreaseContainersResourceResponse increaseContainersResource(
}
@SuppressWarnings("unchecked")
private void changeContainerResourceInternal(
ContainerId containerId, Resource targetResource, boolean increase)
private void changeContainerResourceInternal(ContainerId containerId,
int containerVersion, Resource targetResource, boolean increase)
throws YarnException, IOException {
Container container = context.getContainers().get(containerId);
// Check container existence
@ -1129,7 +1131,7 @@ private void changeContainerResourceInternal(
if (!serviceStopped) {
// Persist container resource change for recovery
this.context.getNMStateStore().storeContainerResourceChanged(
containerId, targetResource);
containerId, containerVersion, targetResource);
getContainersMonitor().handle(
new ChangeMonitoringContainerResourceEvent(
containerId, targetResource));
@ -1364,7 +1366,7 @@ public void handle(ContainerManagerEvent event) {
: containersDecreasedEvent.getContainersToDecrease()) {
try {
changeContainerResourceInternal(container.getId(),
container.getResource(), false);
container.getVersion(), container.getResource(), false);
} catch (YarnException e) {
LOG.error("Unable to decrease container resource", e);
} catch (IOException e) {

View File

@ -102,6 +102,7 @@ public class ContainerImpl implements Container {
private final ContainerId containerId;
private volatile Resource resource;
private final String user;
private int version;
private int exitCode = ContainerExitStatus.INVALID;
private final StringBuilder diagnostics;
private final int diagnosticsMaxSize;
@ -151,6 +152,7 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher,
this.daemonConf = conf;
this.dispatcher = dispatcher;
this.stateStore = context.getNMStateStore();
this.version = containerTokenIdentifier.getVersion();
this.launchContext = launchContext;
if (launchContext != null
&& launchContext.getContainerRetryContext() != null) {
@ -221,6 +223,7 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher,
this.resource = Resource.newInstance(recoveredCapability.getMemorySize(),
recoveredCapability.getVirtualCores());
}
this.version = rcs.getVersion();
this.remainingRetryAttempts = rcs.getRemainingRetryAttempts();
this.workDir = rcs.getWorkDir();
this.logDir = rcs.getLogDir();
@ -519,8 +522,8 @@ public ContainerStatus cloneAndGetContainerStatus() {
public NMContainerStatus getNMContainerStatus() {
this.readLock.lock();
try {
return NMContainerStatus.newInstance(this.containerId, getCurrentState(),
getResource(), diagnostics.toString(), exitCode,
return NMContainerStatus.newInstance(this.containerId, this.version,
getCurrentState(), getResource(), diagnostics.toString(), exitCode,
containerTokenIdentifier.getPriority(),
containerTokenIdentifier.getCreationTime(),
containerTokenIdentifier.getNodeLabelExpression());

View File

@ -104,6 +104,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
private static final String CONTAINERS_KEY_PREFIX =
"ContainerManager/containers/";
private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request";
private static final String CONTAINER_VERSION_KEY_SUFFIX = "/version";
private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics";
private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched";
private static final String CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX =
@ -237,6 +238,8 @@ private RecoveredContainerState loadContainerState(ContainerId containerId,
if (suffix.equals(CONTAINER_REQUEST_KEY_SUFFIX)) {
rcs.startRequest = new StartContainerRequestPBImpl(
StartContainerRequestProto.parseFrom(entry.getValue()));
} else if (suffix.equals(CONTAINER_VERSION_KEY_SUFFIX)) {
rcs.version = Integer.parseInt(asString(entry.getValue()));
} else if (suffix.equals(CONTAINER_DIAGS_KEY_SUFFIX)) {
rcs.diagnostics = asString(entry.getValue());
} else if (suffix.equals(CONTAINER_LAUNCHED_KEY_SUFFIX)) {
@ -266,18 +269,27 @@ private RecoveredContainerState loadContainerState(ContainerId containerId,
}
@Override
public void storeContainer(ContainerId containerId,
public void storeContainer(ContainerId containerId, int containerVersion,
StartContainerRequest startRequest) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("storeContainer: containerId= " + containerId
+ ", startRequest= " + startRequest);
}
String key = CONTAINERS_KEY_PREFIX + containerId.toString()
String keyRequest = CONTAINERS_KEY_PREFIX + containerId.toString()
+ CONTAINER_REQUEST_KEY_SUFFIX;
String keyVersion = CONTAINERS_KEY_PREFIX + containerId.toString()
+ CONTAINER_VERSION_KEY_SUFFIX;
try {
db.put(bytes(key),
((StartContainerRequestPBImpl) startRequest).getProto().toByteArray());
WriteBatch batch = db.createWriteBatch();
try {
batch.put(bytes(keyRequest),
((StartContainerRequestPBImpl) startRequest)
.getProto().toByteArray());
batch.put(bytes(keyVersion), bytes(Integer.toString(containerVersion)));
db.write(batch);
} finally {
batch.close();
}
} catch (DBException e) {
throw new IOException(e);
}
@ -318,18 +330,27 @@ public void storeContainerLaunched(ContainerId containerId)
@Override
public void storeContainerResourceChanged(ContainerId containerId,
Resource capability) throws IOException {
int containerVersion, Resource capability) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("storeContainerResourceChanged: containerId=" + containerId
+ ", capability=" + capability);
}
String key = CONTAINERS_KEY_PREFIX + containerId.toString()
String keyResChng = CONTAINERS_KEY_PREFIX + containerId.toString()
+ CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX;
String keyVersion = CONTAINERS_KEY_PREFIX + containerId.toString()
+ CONTAINER_VERSION_KEY_SUFFIX;
try {
// New value will overwrite old values for the same key
db.put(bytes(key),
((ResourcePBImpl) capability).getProto().toByteArray());
WriteBatch batch = db.createWriteBatch();
try {
// New value will overwrite old values for the same key
batch.put(bytes(keyResChng),
((ResourcePBImpl) capability).getProto().toByteArray());
batch.put(bytes(keyVersion), bytes(Integer.toString(containerVersion)));
db.write(batch);
} finally {
batch.close();
}
} catch (DBException e) {
throw new IOException(e);
}

View File

@ -70,7 +70,7 @@ public List<RecoveredContainerState> loadContainersState()
}
@Override
public void storeContainer(ContainerId containerId,
public void storeContainer(ContainerId containerId, int version,
StartContainerRequest startRequest) throws IOException {
}
@ -86,7 +86,7 @@ public void storeContainerLaunched(ContainerId containerId)
@Override
public void storeContainerResourceChanged(ContainerId containerId,
Resource capability) throws IOException {
int version, Resource capability) throws IOException {
}
@Override

View File

@ -76,6 +76,7 @@ public static class RecoveredContainerState {
private int remainingRetryAttempts = ContainerRetryContext.RETRY_INVALID;
private String workDir;
private String logDir;
int version;
public RecoveredContainerStatus getStatus() {
return status;
@ -93,6 +94,10 @@ public String getDiagnostics() {
return diagnostics;
}
public int getVersion() {
return version;
}
public StartContainerRequest getStartRequest() {
return startRequest;
}
@ -129,6 +134,7 @@ public void setLogDir(String logDir) {
public String toString() {
return new StringBuffer("Status: ").append(getStatus())
.append(", Exit code: ").append(exitCode)
.append(", Version: ").append(version)
.append(", Killed: ").append(getKilled())
.append(", Diagnostics: ").append(getDiagnostics())
.append(", Capability: ").append(getCapability())
@ -305,11 +311,13 @@ public abstract List<RecoveredContainerState> loadContainersState()
/**
* Record a container start request
* @param containerId the container ID
* @param containerVersion the container Version
* @param startRequest the container start request
* @throws IOException
*/
public abstract void storeContainer(ContainerId containerId,
StartContainerRequest startRequest) throws IOException;
int containerVersion, StartContainerRequest startRequest)
throws IOException;
/**
* Record that a container has been launched
@ -322,11 +330,12 @@ public abstract void storeContainerLaunched(ContainerId containerId)
/**
* Record that a container resource has been changed
* @param containerId the container ID
* @param containerVersion the container version
* @param capability the container resource capability
* @throws IOException
*/
public abstract void storeContainerResourceChanged(ContainerId containerId,
Resource capability) throws IOException;
int containerVersion, Resource capability) throws IOException;
/**
* Record that a container has completed

View File

@ -875,7 +875,7 @@ public static NMContainerStatus createNMContainerStatus(int id,
ApplicationAttemptId.newInstance(applicationId, 1);
ContainerId containerId = ContainerId.newContainerId(applicationAttemptId, id);
NMContainerStatus containerReport =
NMContainerStatus.newInstance(containerId, containerState,
NMContainerStatus.newInstance(containerId, 0, containerState,
Resource.newInstance(1024, 1), "recover container", 0,
Priority.newInstance(10), 0);
return containerReport;

View File

@ -251,7 +251,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
String user = "testUser";
ContainerTokenIdentifier containerToken = BuilderUtils
.newContainerTokenIdentifier(BuilderUtils.newContainerToken(
firstContainerID, InetAddress.getByName("localhost")
firstContainerID, 0, InetAddress.getByName("localhost")
.getCanonicalHostName(), 1234, user, resource,
currentTime + 10000, 123, "password".getBytes(), currentTime));
Context context = mock(Context.class);
@ -292,7 +292,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
Resource resource = BuilderUtils.newResource(3, 1);
ContainerTokenIdentifier containerToken = BuilderUtils
.newContainerTokenIdentifier(BuilderUtils.newContainerToken(
secondContainerID, InetAddress.getByName("localhost")
secondContainerID, 0, InetAddress.getByName("localhost")
.getCanonicalHostName(), 1234, user, resource,
currentTime + 10000, 123, "password".getBytes(), currentTime));
Context context = mock(Context.class);
@ -1013,7 +1013,7 @@ public void testRemovePreviousCompletedContainersFromContext() throws Exception
ContainerId cId = ContainerId.newContainerId(appAttemptId, 1);
Token containerToken =
BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
BuilderUtils.newContainerToken(cId, 0, "anyHost", 1234, "anyUser",
BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0);
Container anyCompletedContainer = new ContainerImpl(conf, null,
@ -1035,7 +1035,7 @@ public org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
ContainerId runningContainerId =
ContainerId.newContainerId(appAttemptId, 3);
Token runningContainerToken =
BuilderUtils.newContainerToken(runningContainerId, "anyHost",
BuilderUtils.newContainerToken(runningContainerId, 0, "anyHost",
1234, "anyUser", BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0);
Container runningContainer =
@ -1103,7 +1103,7 @@ public void testKilledQueuedContainers() throws Exception {
ContainerId runningContainerId =
ContainerId.newContainerId(appAttemptId, 1);
Token runningContainerToken =
BuilderUtils.newContainerToken(runningContainerId, "anyHost",
BuilderUtils.newContainerToken(runningContainerId, 0, "anyHost",
1234, "anyUser", BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0);
Container runningContainer =
@ -1131,14 +1131,16 @@ container.ContainerState getContainerState() {
appAttemptId, 2);
ContainerTokenIdentifier killedQueuedContainerTokenId1 = BuilderUtils
.newContainerTokenIdentifier(BuilderUtils.newContainerToken(
killedQueuedContainerId1, "anyHost", 1234, "anyUser", BuilderUtils
.newResource(1024, 1), 0, 123, "password".getBytes(), 0));
killedQueuedContainerId1, 0, "anyHost", 1234, "anyUser",
BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0));
ContainerId killedQueuedContainerId2 = ContainerId.newContainerId(
appAttemptId, 3);
ContainerTokenIdentifier killedQueuedContainerTokenId2 = BuilderUtils
.newContainerTokenIdentifier(BuilderUtils.newContainerToken(
killedQueuedContainerId2, "anyHost", 1234, "anyUser", BuilderUtils
.newResource(1024, 1), 0, 123, "password".getBytes(), 0));
killedQueuedContainerId2, 0, "anyHost", 1234, "anyUser",
BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0));
nm.getNMContext().getQueuingContext().getKilledQueuedContainers().put(
killedQueuedContainerTokenId1, "Queued container killed.");
@ -1214,7 +1216,7 @@ public void testCompletedContainersIsRecentlyStopped() throws Exception {
ApplicationAttemptId.newInstance(appId, 0);
ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
Token containerToken =
BuilderUtils.newContainerToken(containerId, "host", 1234, "user",
BuilderUtils.newContainerToken(containerId, 0, "host", 1234, "user",
BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0);
Container completedContainer = new ContainerImpl(conf, null,
@ -1252,7 +1254,7 @@ public void testCleanedupApplicationContainerCleanup() throws IOException {
ContainerId cId = ContainerId.newContainerId(appAttemptId, 1);
Token containerToken =
BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
BuilderUtils.newContainerToken(cId, 0, "anyHost", 1234, "anyUser",
BuilderUtils.newResource(1024, 1), 0, 123,
"password".getBytes(), 0);
Container anyCompletedContainer = new ContainerImpl(conf, null,

View File

@ -106,6 +106,7 @@
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -297,8 +298,7 @@ public AllocateResponse allocate(AllocateRequest request)
new ArrayList<ContainerStatus>(), containerList,
new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC, 1, null,
new ArrayList<NMToken>(),
new ArrayList<Container>(),
new ArrayList<Container>());
new ArrayList<UpdatedContainer>());
}
@Override

View File

@ -402,7 +402,7 @@ public static Token createContainerToken(ContainerId cId, long rmIdentifier,
LogAggregationContext logAggregationContext, ExecutionType executionType)
throws IOException {
ContainerTokenIdentifier containerTokenIdentifier =
new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
new ContainerTokenIdentifier(cId, 0, nodeId.toString(), user, resource,
System.currentTimeMillis() + 100000L, 123, rmIdentifier,
Priority.newInstance(0), 0, logAggregationContext, null,
ContainerType.TASK, executionType);

View File

@ -125,9 +125,10 @@ public synchronized List<RecoveredContainerState> loadContainersState()
@Override
public synchronized void storeContainer(ContainerId containerId,
StartContainerRequest startRequest) throws IOException {
int version, StartContainerRequest startRequest) throws IOException {
RecoveredContainerState rcs = new RecoveredContainerState();
rcs.startRequest = startRequest;
rcs.version = version;
containerStates.put(containerId, rcs);
}
@ -150,9 +151,11 @@ public synchronized void storeContainerLaunched(ContainerId containerId)
@Override
public synchronized void storeContainerResourceChanged(
ContainerId containerId, Resource capability) throws IOException {
ContainerId containerId, int version, Resource capability)
throws IOException {
RecoveredContainerState rcs = getRecoveredContainerState(containerId);
rcs.capability = capability;
rcs.version = version;
}
@Override

View File

@ -262,11 +262,12 @@ public void testContainerStorage() throws IOException {
StartContainerRequest.newInstance(clc, containerToken);
// store a container and verify recovered
stateStore.storeContainer(containerId, containerReq);
stateStore.storeContainer(containerId, 1, containerReq);
restartStateStore();
recoveredContainers = stateStore.loadContainersState();
assertEquals(1, recoveredContainers.size());
RecoveredContainerState rcs = recoveredContainers.get(0);
assertEquals(1, rcs.getVersion());
assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus());
assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
assertEquals(false, rcs.getKilled());
@ -296,11 +297,13 @@ public void testContainerStorage() throws IOException {
assertEquals(diags.toString(), rcs.getDiagnostics());
// increase the container size, and verify recovered
stateStore.storeContainerResourceChanged(containerId, Resource.newInstance(2468, 4));
stateStore.storeContainerResourceChanged(containerId, 2,
Resource.newInstance(2468, 4));
restartStateStore();
recoveredContainers = stateStore.loadContainersState();
assertEquals(1, recoveredContainers.size());
rcs = recoveredContainers.get(0);
assertEquals(2, rcs.getVersion());
assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
assertEquals(false, rcs.getKilled());

View File

@ -66,7 +66,7 @@ public MockContainer(ApplicationAttemptId appAttemptId,
long currentTime = System.currentTimeMillis();
this.containerTokenIdentifier =
BuilderUtils.newContainerTokenIdentifier(BuilderUtils
.newContainerToken(id, "127.0.0.1", 1234, user,
.newContainerToken(id, 0, "127.0.0.1", 1234, user,
BuilderUtils.newResource(1024, 1), currentTime + 10000, 123,
"password".getBytes(), currentTime));
this.state = ContainerState.NEW;

View File

@ -212,9 +212,9 @@ public boolean isPmemCheckEnabled() {
recordFactory.newRecordInstance(ContainerLaunchContext.class);
long currentTime = System.currentTimeMillis();
Token containerToken =
BuilderUtils.newContainerToken(containerId, "127.0.0.1", 1234, user,
BuilderUtils.newResource(1024, 1), currentTime + 10000L, 123,
"password".getBytes(), currentTime);
BuilderUtils.newContainerToken(containerId, 0, "127.0.0.1", 1234,
user, BuilderUtils.newResource(1024, 1), currentTime + 10000L,
123, "password".getBytes(), currentTime);
Context context = mock(Context.class);
Container container =
new ContainerImpl(conf, dispatcher, launchContext,

View File

@ -55,6 +55,7 @@
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeReport;
@ -66,6 +67,9 @@
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
@ -87,6 +91,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
@ -484,15 +489,6 @@ public AllocateResponse allocate(AllocateRequest request)
throw e;
}
try {
RMServerUtils.increaseDecreaseRequestSanityCheck(rmContext,
request.getIncreaseRequests(), request.getDecreaseRequests(),
maximumCapacity);
} catch (InvalidResourceRequestException e) {
LOG.warn(e);
throw e;
}
// In the case of work-preserving AM restart, it's possible for the
// AM to release containers from the earlier attempt.
if (!app.getApplicationSubmissionContext()
@ -500,11 +496,22 @@ public AllocateResponse allocate(AllocateRequest request)
try {
RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
} catch (InvalidContainerReleaseException e) {
LOG.warn("Invalid container release by application " + appAttemptId, e);
LOG.warn("Invalid container release by application " + appAttemptId,
e);
throw e;
}
}
// Split Update Resource Requests into increase and decrease.
// No Exceptions are thrown here. All update errors are aggregated
// and returned to the AM.
List<UpdateContainerRequest> increaseResourceReqs = new ArrayList<>();
List<UpdateContainerRequest> decreaseResourceReqs = new ArrayList<>();
List<UpdateContainerError> updateContainerErrors =
RMServerUtils.validateAndSplitUpdateResourceRequests(rmContext,
request, maximumCapacity, increaseResourceReqs,
decreaseResourceReqs);
// Send new requests to appAttempt.
Allocation allocation;
RMAppAttemptState state =
@ -519,7 +526,7 @@ public AllocateResponse allocate(AllocateRequest request)
allocation =
this.rScheduler.allocate(appAttemptId, ask, release,
blacklistAdditions, blacklistRemovals,
request.getIncreaseRequests(), request.getDecreaseRequests());
increaseResourceReqs, decreaseResourceReqs);
}
if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
@ -534,6 +541,10 @@ public AllocateResponse allocate(AllocateRequest request)
allocateResponse.setNMTokens(allocation.getNMTokens());
}
// Notify the AM of container update errors
if (!updateContainerErrors.isEmpty()) {
allocateResponse.setUpdateErrors(updateContainerErrors);
}
// update the response with the deltas of node status changes
List<RMNode> updatedNodes = new ArrayList<RMNode>();
if(app.pullRMNodeUpdates(updatedNodes) > 0) {
@ -567,8 +578,23 @@ public AllocateResponse allocate(AllocateRequest request)
allocateResponse.setAvailableResources(allocation.getResourceLimit());
// Handling increased/decreased containers
allocateResponse.setIncreasedContainers(allocation.getIncreasedContainers());
allocateResponse.setDecreasedContainers(allocation.getDecreasedContainers());
List<UpdatedContainer> updatedContainers = new ArrayList<>();
if (allocation.getIncreasedContainers() != null) {
for (Container c : allocation.getIncreasedContainers()) {
updatedContainers.add(
UpdatedContainer.newInstance(
ContainerUpdateType.INCREASE_RESOURCE, c));
}
}
if (allocation.getDecreasedContainers() != null) {
for (Container c : allocation.getDecreasedContainers()) {
updatedContainers.add(
UpdatedContainer.newInstance(
ContainerUpdateType.DECREASE_RESOURCE, c));
}
}
allocateResponse.setUpdatedContainers(updatedContainers);
allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
@ -612,7 +638,7 @@ public AllocateResponse allocate(AllocateRequest request)
return allocateResponse;
}
}
private PreemptionMessage generatePreemptionMessage(Allocation allocation){
PreemptionMessage pMsg = null;
// assemble strict preemption request

View File

@ -6,9 +6,9 @@
* 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
*
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@ -33,28 +33,35 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
import org.apache.hadoop.yarn.exceptions
.InvalidResourceBlacklistRequestException;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
.RMAppAttemptState;
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.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.SchedContainerChangeRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
@ -68,6 +75,18 @@
*/
public class RMServerUtils {
private static final String UPDATE_OUTSTANDING_ERROR =
"UPDATE_OUTSTANDING_ERROR";
private static final String INCORRECT_CONTAINER_VERSION_ERROR =
"INCORRECT_CONTAINER_VERSION_ERROR";
private static final String INVALID_CONTAINER_ID =
"INVALID_CONTAINER_ID";
private static final String RESOURCE_OUTSIDE_ALLOWED_RANGE =
"RESOURCE_OUTSIDE_ALLOWED_RANGE";
protected static final RecordFactory RECORD_FACTORY =
RecordFactoryProvider.getRecordFactory(null);
public static List<RMNode> queryRMNodes(RMContext context,
EnumSet<NodeState> acceptedStates) {
// nodes contains nodes that are NEW, RUNNING, UNHEALTHY or DECOMMISSIONING.
@ -96,6 +115,78 @@ public static List<RMNode> queryRMNodes(RMContext context,
return results;
}
/**
* Check if we have:
* - Request for same containerId and different target resource
* - If targetResources violates maximum/minimumAllocation
* @param rmContext RM context
* @param request Allocate Request
* @param maximumAllocation Maximum Allocation
* @param increaseResourceReqs Increase Resource Request
* @param decreaseResourceReqs Decrease Resource Request
* @return List of container Errors
*/
public static List<UpdateContainerError>
validateAndSplitUpdateResourceRequests(RMContext rmContext,
AllocateRequest request, Resource maximumAllocation,
List<UpdateContainerRequest> increaseResourceReqs,
List<UpdateContainerRequest> decreaseResourceReqs) {
List<UpdateContainerError> errors = new ArrayList<>();
Set<ContainerId> outstandingUpdate = new HashSet<>();
for (UpdateContainerRequest updateReq : request.getUpdateRequests()) {
RMContainer rmContainer = rmContext.getScheduler().getRMContainer(
updateReq.getContainerId());
String msg = null;
if (rmContainer == null) {
msg = INVALID_CONTAINER_ID;
}
// Only allow updates if the requested version matches the current
// version
if (msg == null && updateReq.getContainerVersion() !=
rmContainer.getContainer().getVersion()) {
msg = INCORRECT_CONTAINER_VERSION_ERROR + "|"
+ updateReq.getContainerVersion() + "|"
+ rmContainer.getContainer().getVersion();
}
// No more than 1 container update per request.
if (msg == null &&
outstandingUpdate.contains(updateReq.getContainerId())) {
msg = UPDATE_OUTSTANDING_ERROR;
}
if (msg == null) {
Resource original = rmContainer.getContainer().getResource();
Resource target = updateReq.getCapability();
if (Resources.fitsIn(target, original)) {
// This is a decrease request
if (validateIncreaseDecreaseRequest(rmContext, updateReq,
maximumAllocation, false)) {
decreaseResourceReqs.add(updateReq);
outstandingUpdate.add(updateReq.getContainerId());
} else {
msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
}
} else {
// This is an increase request
if (validateIncreaseDecreaseRequest(rmContext, updateReq,
maximumAllocation, true)) {
increaseResourceReqs.add(updateReq);
outstandingUpdate.add(updateReq.getContainerId());
} else {
msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
}
}
}
if (msg != null) {
UpdateContainerError updateError = RECORD_FACTORY
.newRecordInstance(UpdateContainerError.class);
updateError.setReason(msg);
updateError.setUpdateContainerRequest(updateReq);
errors.add(updateError);
}
}
return errors;
}
/**
* Utility method to validate a list resource requests, by insuring that the
* requested memory/vcore is non-negative and not greater than max
@ -122,8 +213,6 @@ public static void normalizeAndValidateRequests(List<ResourceRequest> ask,
* the queue lock to make sure that the access to container resource is
* atomic. Refer to LeafQueue.decreaseContainer() and
* CapacityScheduelr.updateIncreaseRequests()
*
*
* <pre>
* - Throw exception when any other error happens
* </pre>
@ -145,7 +234,7 @@ public static void checkSchedContainerChangeRequest(
if (increase) {
if (originalResource.getMemorySize() > targetResource.getMemorySize()
|| originalResource.getVirtualCores() > targetResource
.getVirtualCores()) {
.getVirtualCores()) {
String msg =
"Trying to increase a container, but target resource has some"
+ " resource < original resource, target=" + targetResource
@ -156,7 +245,7 @@ public static void checkSchedContainerChangeRequest(
} else {
if (originalResource.getMemorySize() < targetResource.getMemorySize()
|| originalResource.getVirtualCores() < targetResource
.getVirtualCores()) {
.getVirtualCores()) {
String msg =
"Trying to decrease a container, but target resource has "
+ "some resource > original resource, target=" + targetResource
@ -194,112 +283,46 @@ public static void validateBlacklistRequest(
}
}
}
/**
* Check if we have:
* - Request for same containerId and different target resource
* - If targetResources violates maximum/minimumAllocation
*/
public static void increaseDecreaseRequestSanityCheck(RMContext rmContext,
List<ContainerResourceChangeRequest> incRequests,
List<ContainerResourceChangeRequest> decRequests,
Resource maximumAllocation) throws InvalidResourceRequestException {
checkDuplicatedIncreaseDecreaseRequest(incRequests, decRequests);
validateIncreaseDecreaseRequest(rmContext, incRequests, maximumAllocation,
true);
validateIncreaseDecreaseRequest(rmContext, decRequests, maximumAllocation,
false);
}
private static void checkDuplicatedIncreaseDecreaseRequest(
List<ContainerResourceChangeRequest> incRequests,
List<ContainerResourceChangeRequest> decRequests)
throws InvalidResourceRequestException {
String msg = "There're multiple increase or decrease container requests "
+ "for same containerId=";
Set<ContainerId> existedContainerIds = new HashSet<ContainerId>();
if (incRequests != null) {
for (ContainerResourceChangeRequest r : incRequests) {
if (!existedContainerIds.add(r.getContainerId())) {
throw new InvalidResourceRequestException(msg + r.getContainerId());
}
}
}
if (decRequests != null) {
for (ContainerResourceChangeRequest r : decRequests) {
if (!existedContainerIds.add(r.getContainerId())) {
throw new InvalidResourceRequestException(msg + r.getContainerId());
}
}
}
}
// Sanity check and normalize target resource
private static void validateIncreaseDecreaseRequest(RMContext rmContext,
List<ContainerResourceChangeRequest> requests, Resource maximumAllocation,
boolean increase)
throws InvalidResourceRequestException {
if (requests == null) {
return;
private static boolean validateIncreaseDecreaseRequest(RMContext rmContext,
UpdateContainerRequest request, Resource maximumAllocation,
boolean increase) {
if (request.getCapability().getMemorySize() < 0
|| request.getCapability().getMemorySize() > maximumAllocation
.getMemorySize()) {
return false;
}
for (ContainerResourceChangeRequest request : requests) {
if (request.getCapability().getMemorySize() < 0
|| request.getCapability().getMemorySize() > maximumAllocation
.getMemorySize()) {
throw new InvalidResourceRequestException("Invalid "
+ (increase ? "increase" : "decrease") + " request"
+ ", requested memory < 0"
+ ", or requested memory > max configured" + ", requestedMemory="
+ request.getCapability().getMemorySize() + ", maxMemory="
+ maximumAllocation.getMemorySize());
}
if (request.getCapability().getVirtualCores() < 0
|| request.getCapability().getVirtualCores() > maximumAllocation
.getVirtualCores()) {
throw new InvalidResourceRequestException("Invalid "
+ (increase ? "increase" : "decrease") + " request"
+ ", requested virtual cores < 0"
+ ", or requested virtual cores > max configured"
+ ", requestedVirtualCores="
+ request.getCapability().getVirtualCores() + ", maxVirtualCores="
+ maximumAllocation.getVirtualCores());
}
ContainerId containerId = request.getContainerId();
ResourceScheduler scheduler = rmContext.getScheduler();
RMContainer rmContainer = scheduler.getRMContainer(containerId);
if (null == rmContainer) {
String msg =
"Failed to get rmContainer for "
+ (increase ? "increase" : "decrease")
+ " request, with container-id=" + containerId;
throw new InvalidResourceRequestException(msg);
}
ResourceCalculator rc = scheduler.getResourceCalculator();
Resource targetResource = Resources.normalize(rc, request.getCapability(),
scheduler.getMinimumResourceCapability(),
scheduler.getMaximumResourceCapability(),
scheduler.getMinimumResourceCapability());
// Update normalized target resource
request.setCapability(targetResource);
if (request.getCapability().getVirtualCores() < 0
|| request.getCapability().getVirtualCores() > maximumAllocation
.getVirtualCores()) {
return false;
}
ResourceScheduler scheduler = rmContext.getScheduler();
ResourceCalculator rc = scheduler.getResourceCalculator();
Resource targetResource = Resources.normalize(rc, request.getCapability(),
scheduler.getMinimumResourceCapability(),
scheduler.getMaximumResourceCapability(),
scheduler.getMinimumResourceCapability());
// Update normalized target resource
request.setCapability(targetResource);
return true;
}
/**
* It will validate to make sure all the containers belong to correct
* application attempt id. If not then it will throw
* {@link InvalidContainerReleaseException}
*
* @param containerReleaseList
* containers to be released as requested by application master.
* @param appAttemptId
* Application attempt Id
*
* @param containerReleaseList containers to be released as requested by
* application master.
* @param appAttemptId Application attempt Id
* @throws InvalidContainerReleaseException
*/
public static void
validateContainerReleaseRequest(List<ContainerId> containerReleaseList,
ApplicationAttemptId appAttemptId)
throws InvalidContainerReleaseException {
ApplicationAttemptId appAttemptId)
throws InvalidContainerReleaseException {
for (ContainerId cId : containerReleaseList) {
if (!appAttemptId.equals(cId.getApplicationAttemptId())) {
throw new InvalidContainerReleaseException(
@ -321,10 +344,11 @@ public static UserGroupInformation verifyAdminAccess(
/**
* Utility method to verify if the current user has access based on the
* passed {@link AccessControlList}
*
* @param authorizer the {@link AccessControlList} to check against
* @param method the method name to be logged
* @param module like AdminService or NodeLabelManager
* @param LOG the logger to use
* @param method the method name to be logged
* @param module like AdminService or NodeLabelManager
* @param LOG the logger to use
* @return {@link UserGroupInformation} of the current user
* @throws IOException
*/
@ -347,11 +371,11 @@ public static UserGroupInformation verifyAdminAccess(
" to call '" + method + "'");
RMAuditLogger.logFailure(user.getShortUserName(), method, "", module,
RMAuditLogger.AuditConstants.UNAUTHORIZED_USER);
RMAuditLogger.AuditConstants.UNAUTHORIZED_USER);
throw new AccessControlException("User " + user.getShortUserName() +
" doesn't have permission" +
" to call '" + method + "'");
" doesn't have permission" +
" to call '" + method + "'");
}
if (LOG.isTraceEnabled()) {
LOG.trace(method + " invoked by user " + user.getShortUserName());
@ -362,56 +386,56 @@ public static UserGroupInformation verifyAdminAccess(
public static YarnApplicationState createApplicationState(
RMAppState rmAppState) {
switch (rmAppState) {
case NEW:
return YarnApplicationState.NEW;
case NEW_SAVING:
return YarnApplicationState.NEW_SAVING;
case SUBMITTED:
return YarnApplicationState.SUBMITTED;
case ACCEPTED:
return YarnApplicationState.ACCEPTED;
case RUNNING:
return YarnApplicationState.RUNNING;
case FINISHING:
case FINISHED:
return YarnApplicationState.FINISHED;
case KILLED:
return YarnApplicationState.KILLED;
case FAILED:
return YarnApplicationState.FAILED;
default:
throw new YarnRuntimeException("Unknown state passed!");
}
case NEW:
return YarnApplicationState.NEW;
case NEW_SAVING:
return YarnApplicationState.NEW_SAVING;
case SUBMITTED:
return YarnApplicationState.SUBMITTED;
case ACCEPTED:
return YarnApplicationState.ACCEPTED;
case RUNNING:
return YarnApplicationState.RUNNING;
case FINISHING:
case FINISHED:
return YarnApplicationState.FINISHED;
case KILLED:
return YarnApplicationState.KILLED;
case FAILED:
return YarnApplicationState.FAILED;
default:
throw new YarnRuntimeException("Unknown state passed!");
}
}
public static YarnApplicationAttemptState createApplicationAttemptState(
RMAppAttemptState rmAppAttemptState) {
switch (rmAppAttemptState) {
case NEW:
return YarnApplicationAttemptState.NEW;
case SUBMITTED:
return YarnApplicationAttemptState.SUBMITTED;
case SCHEDULED:
return YarnApplicationAttemptState.SCHEDULED;
case ALLOCATED:
return YarnApplicationAttemptState.ALLOCATED;
case LAUNCHED:
return YarnApplicationAttemptState.LAUNCHED;
case ALLOCATED_SAVING:
case LAUNCHED_UNMANAGED_SAVING:
return YarnApplicationAttemptState.ALLOCATED_SAVING;
case RUNNING:
return YarnApplicationAttemptState.RUNNING;
case FINISHING:
return YarnApplicationAttemptState.FINISHING;
case FINISHED:
return YarnApplicationAttemptState.FINISHED;
case KILLED:
return YarnApplicationAttemptState.KILLED;
case FAILED:
return YarnApplicationAttemptState.FAILED;
default:
throw new YarnRuntimeException("Unknown state passed!");
case NEW:
return YarnApplicationAttemptState.NEW;
case SUBMITTED:
return YarnApplicationAttemptState.SUBMITTED;
case SCHEDULED:
return YarnApplicationAttemptState.SCHEDULED;
case ALLOCATED:
return YarnApplicationAttemptState.ALLOCATED;
case LAUNCHED:
return YarnApplicationAttemptState.LAUNCHED;
case ALLOCATED_SAVING:
case LAUNCHED_UNMANAGED_SAVING:
return YarnApplicationAttemptState.ALLOCATED_SAVING;
case RUNNING:
return YarnApplicationAttemptState.RUNNING;
case FINISHING:
return YarnApplicationAttemptState.FINISHING;
case FINISHED:
return YarnApplicationAttemptState.FINISHED;
case KILLED:
return YarnApplicationAttemptState.KILLED;
case FAILED:
return YarnApplicationAttemptState.FAILED;
default:
throw new YarnRuntimeException("Unknown state passed!");
}
}
@ -420,13 +444,12 @@ public static YarnApplicationAttemptState createApplicationAttemptState(
* a return value when a valid report cannot be found.
*/
public static final ApplicationResourceUsageReport
DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
BuilderUtils.newApplicationResourceUsageReport(-1, -1,
Resources.createResource(-1, -1), Resources.createResource(-1, -1),
Resources.createResource(-1, -1), 0, 0);
/**
* Find all configs whose name starts with
* YarnConfiguration.RM_PROXY_USER_PREFIX, and add a record for each one by
@ -438,7 +461,8 @@ public static void processRMProxyUsersConf(Configuration conf) {
String propName = entry.getKey();
if (propName.startsWith(YarnConfiguration.RM_PROXY_USER_PREFIX)) {
rmProxyUsers.put(ProxyUsers.CONF_HADOOP_PROXYUSER + "." +
propName.substring(YarnConfiguration.RM_PROXY_USER_PREFIX.length()),
propName.substring(YarnConfiguration.RM_PROXY_USER_PREFIX
.length()),
entry.getValue());
}
}

View File

@ -40,7 +40,6 @@
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
@ -48,6 +47,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -434,6 +434,7 @@ private RMContainer recoverAndCreateContainer(NMContainerStatus status,
Container.newInstance(status.getContainerId(), node.getNodeID(),
node.getHttpAddress(), status.getAllocatedResource(),
status.getPriority(), null);
container.setVersion(status.getVersion());
ApplicationAttemptId attemptId =
container.getId().getApplicationAttemptId();
RMContainer rmContainer =
@ -572,7 +573,7 @@ protected void releaseContainers(List<ContainerId> containers,
}
protected void decreaseContainers(
List<ContainerResourceChangeRequest> decreaseRequests,
List<UpdateContainerRequest> decreaseRequests,
SchedulerApplicationAttempt attempt) {
if (null == decreaseRequests || decreaseRequests.isEmpty()) {
return;
@ -745,7 +746,7 @@ public synchronized void setClusterMaxPriority(Configuration conf)
/**
* Sanity check increase/decrease request, and return
* SchedulerContainerResourceChangeRequest according to given
* ContainerResourceChangeRequest.
* UpdateContainerRequest.
*
* <pre>
* - Returns non-null value means validation succeeded
@ -753,7 +754,7 @@ public synchronized void setClusterMaxPriority(Configuration conf)
* </pre>
*/
private SchedContainerChangeRequest createSchedContainerChangeRequest(
ContainerResourceChangeRequest request, boolean increase)
UpdateContainerRequest request, boolean increase)
throws YarnException {
ContainerId containerId = request.getContainerId();
RMContainer rmContainer = getRMContainer(containerId);
@ -772,11 +773,11 @@ private SchedContainerChangeRequest createSchedContainerChangeRequest(
protected List<SchedContainerChangeRequest>
createSchedContainerChangeRequests(
List<ContainerResourceChangeRequest> changeRequests,
List<UpdateContainerRequest> changeRequests,
boolean increase) {
List<SchedContainerChangeRequest> schedulerChangeRequests =
new ArrayList<SchedContainerChangeRequest>();
for (ContainerResourceChangeRequest r : changeRequests) {
for (UpdateContainerRequest r : changeRequests) {
SchedContainerChangeRequest sr = null;
try {
sr = createSchedContainerChangeRequest(r, increase);

View File

@ -27,7 +27,7 @@
import org.apache.hadoop.yarn.util.resource.Resources;
/**
* This is ContainerResourceChangeRequest in scheduler side, it contains some
* This is UpdateContainerRequest in scheduler side, it contains some
* pointers to runtime objects like RMContainer, SchedulerNode, etc. This will
* be easier for scheduler making decision.
*/

View File

@ -530,6 +530,9 @@ private Container updateContainerAndNMToken(RMContainer rmContainer,
boolean newContainer, boolean increasedContainer) {
Container container = rmContainer.getContainer();
ContainerType containerType = ContainerType.TASK;
if (!newContainer) {
container.setVersion(container.getVersion() + 1);
}
// The working knowledge is that masterContainer for AM is null as it
// itself is the master container.
if (isWaitingForAMContainer()) {
@ -538,10 +541,11 @@ private Container updateContainerAndNMToken(RMContainer rmContainer,
try {
// create container token and NMToken altogether.
container.setContainerToken(rmContext.getContainerTokenSecretManager()
.createContainerToken(container.getId(), container.getNodeId(),
getUser(), container.getResource(), container.getPriority(),
rmContainer.getCreationTime(), this.logAggregationContext,
rmContainer.getNodeLabelExpression(), containerType));
.createContainerToken(container.getId(), container.getVersion(),
container.getNodeId(), getUser(), container.getResource(),
container.getPriority(), rmContainer.getCreationTime(),
this.logAggregationContext, rmContainer.getNodeLabelExpression(),
containerType));
NMToken nmToken =
rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
getApplicationAttemptId(), container);

View File

@ -35,7 +35,6 @@
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.QueueACL;
@ -43,6 +42,7 @@
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -143,8 +143,8 @@ public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
Allocation allocate(ApplicationAttemptId appAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests);
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests);
/**
* Get node resource usage report.

View File

@ -51,7 +51,6 @@
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.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
@ -63,6 +62,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -917,7 +917,7 @@ private synchronized void doneApplicationAttempt(
// SchedContainerChangeRequest
// 2. Deadlock with the scheduling thread.
private LeafQueue updateIncreaseRequests(
List<ContainerResourceChangeRequest> increaseRequests,
List<UpdateContainerRequest> increaseRequests,
FiCaSchedulerApp app) {
if (null == increaseRequests || increaseRequests.isEmpty()) {
return null;
@ -945,8 +945,8 @@ private LeafQueue updateIncreaseRequests(
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests) {
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
if (application == null) {

View File

@ -39,7 +39,6 @@
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.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
@ -50,6 +49,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@ -942,8 +942,8 @@ private synchronized void removeNode(RMNode rmNode) {
public Allocation allocate(ApplicationAttemptId appAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests) {
// Make sure this application exists
FSAppAttempt application = getSchedulerApp(appAttemptId);

View File

@ -40,7 +40,6 @@
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.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
@ -52,6 +51,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
@ -330,8 +330,8 @@ public synchronized void setRMContext(RMContext rmContext) {
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) {
List<UpdateContainerRequest> increaseRequests,
List<UpdateContainerRequest> decreaseRequests) {
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
if (application == null) {
LOG.error("Calling allocate on removed " +

View File

@ -26,6 +26,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.LogAggregationContext;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
@ -168,39 +169,43 @@ public void run() {
/**
* Helper function for creating ContainerTokens
*
* @param containerId
* @param nodeId
* @param appSubmitter
* @param capability
* @param priority
* @param createTime
* @param containerId Container Id
* @param containerVersion Container Version
* @param nodeId Node Id
* @param appSubmitter App Submitter
* @param capability Capability
* @param priority Priority
* @param createTime Create Time
* @return the container-token
*/
public Token createContainerToken(ContainerId containerId, NodeId nodeId,
String appSubmitter, Resource capability, Priority priority,
long createTime) {
return createContainerToken(containerId, nodeId, appSubmitter, capability,
priority, createTime, null, null, ContainerType.TASK);
public Token createContainerToken(ContainerId containerId,
int containerVersion, NodeId nodeId, String appSubmitter,
Resource capability, Priority priority, long createTime) {
return createContainerToken(containerId, containerVersion, nodeId,
appSubmitter, capability, priority, createTime,
null, null, ContainerType.TASK);
}
/**
* Helper function for creating ContainerTokens
*
* @param containerId
* @param nodeId
* @param appSubmitter
* @param capability
* @param priority
* @param createTime
* @param logAggregationContext
* @param nodeLabelExpression
* @param containerType
* @param containerId Container Id
* @param containerVersion Container version
* @param nodeId Node Id
* @param appSubmitter App Submitter
* @param capability Capability
* @param priority Priority
* @param createTime Create Time
* @param logAggregationContext Log Aggregation Context
* @param nodeLabelExpression Node Label Expression
* @param containerType Container Type
* @return the container-token
*/
public Token createContainerToken(ContainerId containerId, NodeId nodeId,
String appSubmitter, Resource capability, Priority priority,
long createTime, LogAggregationContext logAggregationContext,
String nodeLabelExpression, ContainerType containerType) {
public Token createContainerToken(ContainerId containerId,
int containerVersion, NodeId nodeId, String appSubmitter,
Resource capability, Priority priority, long createTime,
LogAggregationContext logAggregationContext, String nodeLabelExpression,
ContainerType containerType) {
byte[] password;
ContainerTokenIdentifier tokenIdentifier;
long expiryTimeStamp =
@ -210,11 +215,12 @@ public Token createContainerToken(ContainerId containerId, NodeId nodeId,
this.readLock.lock();
try {
tokenIdentifier =
new ContainerTokenIdentifier(containerId, nodeId.toString(),
appSubmitter, capability, expiryTimeStamp, this.currentMasterKey
.getMasterKey().getKeyId(),
ResourceManager.getClusterTimeStamp(), priority, createTime,
logAggregationContext, nodeLabelExpression, containerType);
new ContainerTokenIdentifier(containerId, containerVersion,
nodeId.toString(), appSubmitter, capability, expiryTimeStamp,
this.currentMasterKey.getMasterKey().getKeyId(),
ResourceManager.getClusterTimeStamp(), priority, createTime,
logAggregationContext, nodeLabelExpression, containerType,
ExecutionType.GUARANTEED);
password = this.createPassword(tokenIdentifier);
} finally {

View File

@ -34,12 +34,12 @@
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@ -245,10 +245,9 @@ public AllocateResponse allocate(
}
public AllocateResponse sendContainerResizingRequest(
List<ContainerResourceChangeRequest> increaseRequests,
List<ContainerResourceChangeRequest> decreaseRequests) throws Exception {
List<UpdateContainerRequest> updateRequests) throws Exception {
final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null,
null, increaseRequests, decreaseRequests);
null, updateRequests);
return allocate(req);
}

View File

@ -386,9 +386,10 @@ public void testAppCleanupWhenRMRestartedBeforeAppFinished() throws Exception {
// nm1/nm2 register to rm2, and do a heartbeat
nm1.setResourceTrackerService(rm2.getResourceTrackerService());
nm1.registerNode(Arrays.asList(NMContainerStatus.newInstance(
ContainerId.newContainerId(am0.getApplicationAttemptId(), 1),
ContainerState.COMPLETE, Resource.newInstance(1024, 1), "", 0,
Priority.newInstance(0), 1234)), Arrays.asList(app0.getApplicationId()));
ContainerId.newContainerId(am0.getApplicationAttemptId(), 1), 0,
ContainerState.COMPLETE, Resource.newInstance(1024, 1), "", 0,
Priority.newInstance(0), 1234)),
Arrays.asList(app0.getApplicationId()));
nm2.setResourceTrackerService(rm2.getResourceTrackerService());
nm2.registerNode(Arrays.asList(app0.getApplicationId()));
@ -598,7 +599,7 @@ public static NMContainerStatus createNMContainerStatus(
int memory) {
ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
NMContainerStatus containerReport =
NMContainerStatus.newInstance(containerId, containerState,
NMContainerStatus.newInstance(containerId, 0, containerState,
Resource.newInstance(memory, 1), "recover container", 0,
Priority.newInstance(0), 0);
return containerReport;

View File

@ -35,16 +35,16 @@
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@ -383,57 +383,47 @@ public void testInvalidIncreaseDecreaseRequest() throws Exception {
// Ask for a normal increase should be successfull
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest.newInstance(
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
Resources.createResource(2048))), null);
UpdateContainerRequest.newInstance(
0, ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(2048), null)));
// Target resource is negative, should fail
boolean exceptionCaught = false;
try {
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest.newInstance(
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
Resources.createResource(-1))), null);
} catch (InvalidResourceRequestException e) {
// This is expected
exceptionCaught = true;
}
Assert.assertTrue(exceptionCaught);
AllocateResponse response =
am1.sendContainerResizingRequest(Arrays.asList(
UpdateContainerRequest.newInstance(0,
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(-1), null)));
Assert.assertEquals(1, response.getUpdateErrors().size());
Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
response.getUpdateErrors().get(0).getReason());
// Target resource is more than maxAllocation, should fail
try {
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest.newInstance(
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
Resources
.add(registerResponse.getMaximumResourceCapability(),
Resources.createResource(1)))), null);
} catch (InvalidResourceRequestException e) {
// This is expected
exceptionCaught = true;
}
response = am1.sendContainerResizingRequest(Arrays.asList(
UpdateContainerRequest.newInstance(0,
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
ContainerUpdateType.INCREASE_RESOURCE,
Resources.add(
registerResponse.getMaximumResourceCapability(),
Resources.createResource(1)), null)));
Assert.assertEquals(1, response.getUpdateErrors().size());
Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
response.getUpdateErrors().get(0).getReason());
Assert.assertTrue(exceptionCaught);
// Contains multiple increase/decrease requests for same contaienrId
try {
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest.newInstance(
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
Resources
.add(registerResponse.getMaximumResourceCapability(),
Resources.createResource(1)))), Arrays.asList(
ContainerResourceChangeRequest.newInstance(
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
Resources
.add(registerResponse.getMaximumResourceCapability(),
Resources.createResource(1)))));
} catch (InvalidResourceRequestException e) {
// This is expected
exceptionCaught = true;
}
Assert.assertTrue(exceptionCaught);
response = am1.sendContainerResizingRequest(Arrays.asList(
UpdateContainerRequest.newInstance(0,
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(2048, 4), null),
UpdateContainerRequest.newInstance(0,
ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
ContainerUpdateType.DECREASE_RESOURCE,
Resources.createResource(1024, 1), null)));
Assert.assertEquals(1, response.getUpdateErrors().size());
Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
response.getUpdateErrors().get(0).getReason());
} finally {
if (rm != null) {
rm.close();

View File

@ -2029,7 +2029,7 @@ public static NMContainerStatus createNMContainerStatus(
String nodeLabelExpression) {
ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
NMContainerStatus containerReport =
NMContainerStatus.newInstance(containerId, containerState,
NMContainerStatus.newInstance(containerId, 0, containerState,
Resource.newInstance(1024, 1), "recover container", 0,
Priority.newInstance(0), 0, nodeLabelExpression);
return containerReport;

View File

@ -1041,7 +1041,7 @@ public void testHandleContainerStatusInvalidCompletions() throws Exception {
NMContainerStatus report =
NMContainerStatus.newInstance(
ContainerId.newContainerId(
ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1), 0,
ContainerState.COMPLETE, Resource.newInstance(1024, 1),
"Dummy Completed", 0, Priority.newInstance(10), 1234);
rm.getResourceTrackerService().handleNMContainerStatus(report, null);
@ -1052,7 +1052,7 @@ public void testHandleContainerStatusInvalidCompletions() throws Exception {
(RMAppAttemptImpl) app.getCurrentAppAttempt();
currentAttempt.setMasterContainer(null);
report = NMContainerStatus.newInstance(
ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0),
ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0), 0,
ContainerState.COMPLETE, Resource.newInstance(1024, 1),
"Dummy Completed", 0, Priority.newInstance(10), 1234);
rm.getResourceTrackerService().handleNMContainerStatus(report, null);
@ -1064,7 +1064,7 @@ public void testHandleContainerStatusInvalidCompletions() throws Exception {
// Case 2.1: AppAttemptId is null
report = NMContainerStatus.newInstance(
ContainerId.newContainerId(
ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1), 0,
ContainerState.COMPLETE, Resource.newInstance(1024, 1),
"Dummy Completed", 0, Priority.newInstance(10), 1234);
try {
@ -1079,7 +1079,7 @@ public void testHandleContainerStatusInvalidCompletions() throws Exception {
(RMAppAttemptImpl) app.getCurrentAppAttempt();
currentAttempt.setMasterContainer(null);
report = NMContainerStatus.newInstance(
ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0),
ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0), 0,
ContainerState.COMPLETE, Resource.newInstance(1024, 1),
"Dummy Completed", 0, Priority.newInstance(10), 1234);
try {

View File

@ -58,9 +58,9 @@
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.Priority;
@ -70,6 +70,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
@ -3232,9 +3233,10 @@ protected RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 1GB to 3GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(3 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null)));
FiCaSchedulerApp app = getFiCaSchedulerApp(rm, app1.getApplicationId());
@ -3246,11 +3248,14 @@ protected RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change containerId2 (2G -> 3G) and containerId3 (2G -> 5G)
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId2, Resources.createResource(3 * GB)),
ContainerResourceChangeRequest
.newInstance(containerId3, Resources.createResource(5 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null),
UpdateContainerRequest
.newInstance(0, containerId3,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(5 * GB), null)));
Assert.assertEquals(6 * GB,
app.getAppAttemptResourceUsage().getPending().getMemorySize());
@ -3261,13 +3266,18 @@ protected RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change containerId1 (1G->3G), containerId2 (2G -> 4G) and
// containerId3 (2G -> 2G)
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(3 * GB)),
ContainerResourceChangeRequest
.newInstance(containerId2, Resources.createResource(4 * GB)),
ContainerResourceChangeRequest
.newInstance(containerId3, Resources.createResource(2 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null),
UpdateContainerRequest
.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(4 * GB), null),
UpdateContainerRequest
.newInstance(0, containerId3,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(2 * GB), null)));
Assert.assertEquals(4 * GB,
app.getAppAttemptResourceUsage().getPending().getMemorySize());
checkPendingResource(rm, "a1", 4 * GB, null);

View File

@ -289,13 +289,14 @@ protected RMContainerTokenSecretManager createContainerTokenSecretManager(
@Override
public Token createContainerToken(ContainerId containerId,
NodeId nodeId, String appSubmitter, Resource capability,
Priority priority, long createTime,
LogAggregationContext logAggregationContext, String nodeLabelExp, ContainerType containerType) {
int containerVersion, NodeId nodeId, String appSubmitter,
Resource capability, Priority priority, long createTime,
LogAggregationContext logAggregationContext, String nodeLabelExp,
ContainerType containerType) {
numRetries++;
return super.createContainerToken(containerId, nodeId, appSubmitter,
capability, priority, createTime, logAggregationContext,
nodeLabelExp, containerType);
return super.createContainerToken(containerId, containerVersion,
nodeId, appSubmitter, capability, priority, createTime,
logAggregationContext, nodeLabelExp, containerType);
}
};
}

View File

@ -30,10 +30,12 @@
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
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.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@ -84,7 +86,7 @@ class MyScheduler extends CapacityScheduler {
@Override
protected void decreaseContainers(
List<ContainerResourceChangeRequest> decreaseRequests,
List<UpdateContainerRequest> decreaseRequests,
SchedulerApplicationAttempt attempt) {
try {
Thread.sleep(1000);
@ -138,9 +140,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
sentRMContainerLaunched(rm1, containerId1);
// am1 asks to change its AM container from 1GB to 3GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(3 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null)));
FiCaSchedulerApp app = TestUtils.getFiCaSchedulerApp(
rm1, app1.getApplicationId());
@ -195,9 +198,11 @@ public RMNodeLabelsManager createNodeLabelManager() {
sentRMContainerLaunched(rm1, containerId1);
// am1 asks to change its AM container from 1GB to 3GB
AllocateResponse response = am1.sendContainerResizingRequest(null, Arrays
.asList(ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(1 * GB))));
AllocateResponse response = am1.sendContainerResizingRequest(Arrays
.asList(UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.DECREASE_RESOURCE,
Resources.createResource(1 * GB), null)));
verifyContainerDecreased(response, containerId1, 1 * GB);
checkUsedResource(rm1, "default", 1 * GB, null);
@ -266,9 +271,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 1GB to 3GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(7 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(7 * GB), null)));
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -367,9 +373,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change container2 from 2GB to 8GB, which will exceed user
// limit
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId2, Resources.createResource(8 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(8 * GB), null)));
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -447,9 +454,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 1GB to 3GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(7 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(7 * GB), null)));
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -487,9 +495,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 1G to 1G (cancel the increase
// request actually)
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(1 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(1 * GB), null)));
// Trigger a node heartbeat..
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
@ -559,9 +568,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 2GB to 8GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(8 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(8 * GB), null)));
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -597,9 +607,11 @@ public RMNodeLabelsManager createNodeLabelManager() {
// request, make target_capacity=existing_capacity)
am1.allocate(null, Arrays.asList(containerId2));
// am1 asks to change its AM container from 2G to 1G (decrease)
am1.sendContainerResizingRequest(null, Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(1 * GB))));
am1.sendContainerResizingRequest(Arrays.asList(
UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(1 * GB), null)));
// Trigger a node heartbeat..
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
@ -668,9 +680,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 2GB to 8GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId2, Resources.createResource(8 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(8 * GB), null)));
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -765,9 +778,10 @@ public RMNodeLabelsManager createNodeLabelManager() {
// am1 asks to change its AM container from 2GB to 8GB
am1.sendContainerResizingRequest(Arrays.asList(
ContainerResourceChangeRequest
.newInstance(containerId2, Resources.createResource(8 * GB))),
null);
UpdateContainerRequest
.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(8 * GB), null)));
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -883,14 +897,16 @@ public RMNodeLabelsManager createNodeLabelManager() {
allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6);
// am1 asks to change its container[2-7] from 1G to 2G
List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
for (int cId = 2; cId <= 7; cId++) {
ContainerId containerId =
ContainerId.newContainerId(am1.getApplicationAttemptId(), cId);
increaseRequests.add(ContainerResourceChangeRequest
.newInstance(containerId, Resources.createResource(2 * GB)));
increaseRequests.add(UpdateContainerRequest
.newInstance(0, containerId,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(2 * GB), null));
}
am1.sendContainerResizingRequest(increaseRequests, null);
am1.sendContainerResizingRequest(increaseRequests);
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -904,7 +920,7 @@ public RMNodeLabelsManager createNodeLabelManager() {
// earlier allocated)
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
AllocateResponse allocateResponse = am1.allocate(null, null);
Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size());
Assert.assertEquals(3, allocateResponse.getUpdatedContainers().size());
verifyContainerIncreased(allocateResponse,
ContainerId.newContainerId(attemptId, 4), 2 * GB);
verifyContainerIncreased(allocateResponse,
@ -964,14 +980,16 @@ public RMNodeLabelsManager createNodeLabelManager() {
allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6);
// am1 asks to change its container[2-7] from 1G to 2G
List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
for (int cId = 2; cId <= 7; cId++) {
ContainerId containerId =
ContainerId.newContainerId(am1.getApplicationAttemptId(), cId);
increaseRequests.add(ContainerResourceChangeRequest
.newInstance(containerId, Resources.createResource(2 * GB)));
increaseRequests.add(UpdateContainerRequest
.newInstance(0, containerId,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(2 * GB), null));
}
am1.sendContainerResizingRequest(increaseRequests, null);
am1.sendContainerResizingRequest(increaseRequests);
checkPendingResource(rm1, "default", 6 * GB, null);
Assert.assertEquals(6 * GB,
@ -985,7 +1003,7 @@ public RMNodeLabelsManager createNodeLabelManager() {
// earlier allocated)
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
AllocateResponse allocateResponse = am1.allocate(null, null);
Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size());
Assert.assertEquals(3, allocateResponse.getUpdatedContainers().size());
verifyContainerIncreased(allocateResponse,
ContainerId.newContainerId(attemptId, 4), 2 * GB);
verifyContainerIncreased(allocateResponse,
@ -1046,9 +1064,11 @@ public ResourceScheduler createScheduler() {
nm.nodeHeartbeat(true);
// *In the mean time*, am1 asks to decrease its AM container resource from
// 3GB to 1GB
AllocateResponse response = am1.sendContainerResizingRequest(null,
Collections.singletonList(ContainerResourceChangeRequest
.newInstance(containerId1, Resources.createResource(GB))));
AllocateResponse response = am1.sendContainerResizingRequest(
Collections.singletonList(UpdateContainerRequest
.newInstance(0, containerId1,
ContainerUpdateType.DECREASE_RESOURCE,
Resources.createResource(GB), null)));
// verify that the containe resource is decreased
verifyContainerDecreased(response, containerId1, GB);
@ -1077,12 +1097,16 @@ private void checkUsedResource(MockRM rm, String queueName, int memory,
private void verifyContainerIncreased(AllocateResponse response,
ContainerId containerId, int mem) {
List<Container> increasedContainers = response.getIncreasedContainers();
List<UpdatedContainer> increasedContainers =
response.getUpdatedContainers();
boolean found = false;
for (Container c : increasedContainers) {
if (c.getId().equals(containerId)) {
for (UpdatedContainer c : increasedContainers) {
if (c.getContainer().getId().equals(containerId)) {
found = true;
Assert.assertEquals(mem, c.getResource().getMemorySize());
Assert.assertEquals(ContainerUpdateType.INCREASE_RESOURCE,
c.getUpdateType());
Assert.assertEquals(mem,
c.getContainer().getResource().getMemorySize());
}
}
if (!found) {
@ -1092,12 +1116,16 @@ private void verifyContainerIncreased(AllocateResponse response,
private void verifyContainerDecreased(AllocateResponse response,
ContainerId containerId, int mem) {
List<Container> decreasedContainers = response.getDecreasedContainers();
List<UpdatedContainer> decreasedContainers =
response.getUpdatedContainers();
boolean found = false;
for (Container c : decreasedContainers) {
if (c.getId().equals(containerId)) {
for (UpdatedContainer c : decreasedContainers) {
if (c.getContainer().getId().equals(containerId)) {
found = true;
Assert.assertEquals(mem, c.getResource().getMemorySize());
Assert.assertEquals(ContainerUpdateType.DECREASE_RESOURCE,
c.getUpdateType());
Assert.assertEquals(mem,
c.getContainer().getResource().getMemorySize());
}
}
if (!found) {

View File

@ -21,10 +21,12 @@
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@ -109,8 +111,9 @@ public void testContainerIsRemovedFromAllocationExpirer()
rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
// am1 asks to increase containerId2 from 1GB to 3GB
am1.sendContainerResizingRequest(Collections.singletonList(
ContainerResourceChangeRequest.newInstance(
containerId2, Resources.createResource(3 * GB))), null);
UpdateContainerRequest.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null)));
// Kick off scheduling and sleep for 1 second;
nm1.nodeHeartbeat(true);
Thread.sleep(1000);
@ -180,8 +183,9 @@ public void testContainerIncreaseAllocationExpiration()
rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
// am1 asks to increase containerId2 from 1GB to 3GB
am1.sendContainerResizingRequest(Collections.singletonList(
ContainerResourceChangeRequest.newInstance(
containerId2, Resources.createResource(3 * GB))), null);
UpdateContainerRequest.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null)));
// Kick off scheduling and wait for 1 second;
nm1.nodeHeartbeat(true);
Thread.sleep(1000);
@ -249,8 +253,9 @@ public void testConsecutiveContainerIncreaseAllocationExpiration()
rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
// am1 asks to change containerId2 from 1GB to 3GB
am1.sendContainerResizingRequest(Collections.singletonList(
ContainerResourceChangeRequest.newInstance(
containerId2, Resources.createResource(3 * GB))), null);
UpdateContainerRequest.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(3 * GB), null)));
// Kick off scheduling and sleep for 1 second to
// make sure the allocation is done
nm1.nodeHeartbeat(true);
@ -261,10 +266,23 @@ public void testConsecutiveContainerIncreaseAllocationExpiration()
Resource resource1 = Resources.clone(
rm1.getResourceScheduler().getRMContainer(containerId2)
.getAllocatedResource());
// This should not work, since the container version is wrong
AllocateResponse response = am1.sendContainerResizingRequest(Collections
.singletonList(
UpdateContainerRequest.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(5 * GB), null)));
List<UpdateContainerError> updateErrors = response.getUpdateErrors();
Assert.assertEquals(1, updateErrors.size());
Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR|0|1",
updateErrors.get(0).getReason());
// am1 asks to change containerId2 from 3GB to 5GB
am1.sendContainerResizingRequest(Collections.singletonList(
ContainerResourceChangeRequest.newInstance(
containerId2, Resources.createResource(5 * GB))), null);
UpdateContainerRequest.newInstance(1, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(5 * GB), null)));
// Kick off scheduling and sleep for 1 second to
// make sure the allocation is done
nm1.nodeHeartbeat(true);
@ -362,30 +380,36 @@ public void testDecreaseAfterIncreaseWithAllocationExpiration()
rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
rm1.waitForState(nm1, containerId4, RMContainerState.RUNNING);
// am1 asks to change containerId2 and containerId3 from 1GB to 3GB
List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
increaseRequests.add(ContainerResourceChangeRequest.newInstance(
containerId2, Resources.createResource(6 * GB)));
increaseRequests.add(ContainerResourceChangeRequest.newInstance(
containerId3, Resources.createResource(6 * GB)));
increaseRequests.add(ContainerResourceChangeRequest.newInstance(
containerId4, Resources.createResource(6 * GB)));
am1.sendContainerResizingRequest(increaseRequests, null);
List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
increaseRequests.add(UpdateContainerRequest.newInstance(0, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(6 * GB), null));
increaseRequests.add(UpdateContainerRequest.newInstance(0, containerId3,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(6 * GB), null));
increaseRequests.add(UpdateContainerRequest.newInstance(0, containerId4,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(6 * GB), null));
am1.sendContainerResizingRequest(increaseRequests);
nm1.nodeHeartbeat(true);
Thread.sleep(1000);
// Start container increase allocation expirer
am1.allocate(null, null);
// Decrease containers
List<ContainerResourceChangeRequest> decreaseRequests = new ArrayList<>();
decreaseRequests.add(ContainerResourceChangeRequest.newInstance(
containerId2, Resources.createResource(2 * GB)));
decreaseRequests.add(ContainerResourceChangeRequest.newInstance(
containerId3, Resources.createResource(4 * GB)));
decreaseRequests.add(ContainerResourceChangeRequest.newInstance(
containerId4, Resources.createResource(4 * GB)));
List<UpdateContainerRequest> decreaseRequests = new ArrayList<>();
decreaseRequests.add(UpdateContainerRequest.newInstance(1, containerId2,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(2 * GB), null));
decreaseRequests.add(UpdateContainerRequest.newInstance(1, containerId3,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(4 * GB), null));
decreaseRequests.add(UpdateContainerRequest.newInstance(1, containerId4,
ContainerUpdateType.INCREASE_RESOURCE,
Resources.createResource(4 * GB), null));
AllocateResponse response =
am1.sendContainerResizingRequest(null, decreaseRequests);
am1.sendContainerResizingRequest(decreaseRequests);
// Verify containers are decreased in scheduler
Assert.assertEquals(3, response.getDecreasedContainers().size());
Assert.assertEquals(3, response.getUpdatedContainers().size());
// Use the token for containerId4 on NM (6G). This should set the last
// confirmed resource to 4G, and cancel the allocation expirer
nm1.containerIncreaseStatus(getContainer(

View File

@ -220,7 +220,7 @@ private void testNMTokens(Configuration conf) throws Exception {
org.apache.hadoop.yarn.api.records.Token validContainerToken =
containerTokenSecretManager.createContainerToken(validContainerId,
validNode, user, r, Priority.newInstance(10), 1234);
0, validNode, user, r, Priority.newInstance(10), 1234);
ContainerTokenIdentifier identifier =
BuilderUtils.newContainerTokenIdentifier(validContainerToken);
Assert.assertEquals(Priority.newInstance(10), identifier.getPriority());
@ -277,7 +277,7 @@ private void testNMTokens(Configuration conf) throws Exception {
4 * 60 * 1000);
validContainerToken =
containerTokenSecretManager.createContainerToken(validContainerId,
validNode, user, r, Priority.newInstance(0), 0);
0, validNode, user, r, Priority.newInstance(0), 0);
Assert.assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
validContainerToken, validNMToken, false).isEmpty());
Assert.assertTrue(nmTokenSecretManagerNM
@ -293,7 +293,7 @@ private void testNMTokens(Configuration conf) throws Exception {
org.apache.hadoop.yarn.api.records.Token validContainerToken2 =
containerTokenSecretManager.createContainerToken(validContainerId2,
validNode, user, r, Priority.newInstance(0), 0);
0, validNode, user, r, Priority.newInstance(0), 0);
org.apache.hadoop.yarn.api.records.Token validNMToken2 =
nmTokenSecretManagerRM.createNMToken(validAppAttemptId2, validNode, user);
@ -379,7 +379,7 @@ private void testNMTokens(Configuration conf) throws Exception {
.createNMToken(validAppAttemptId, validNode, user);
org.apache.hadoop.yarn.api.records.Token newContainerToken =
containerTokenSecretManager.createContainerToken(
ContainerId.newContainerId(attempt2, 1), validNode, user, r,
ContainerId.newContainerId(attempt2, 1), 0, validNode, user, r,
Priority.newInstance(0), 0);
Assert.assertTrue(testStartContainer(rpc, attempt2, validNode,
newContainerToken, attempt1NMToken, false).isEmpty());
@ -639,7 +639,7 @@ private void testContainerToken(Configuration conf) throws IOException,
Token containerToken =
containerTokenSecretManager.createContainerToken(
cId, nodeId, user, r, Priority.newInstance(0), 0);
cId, 0, nodeId, user, r, Priority.newInstance(0), 0);
ContainerTokenIdentifier containerTokenIdentifier =
getContainerTokenIdentifierFromToken(containerToken);
@ -672,8 +672,8 @@ private void testContainerToken(Configuration conf) throws IOException,
ContainerId cId2 = ContainerId.newContainerId(appAttemptId, 1);
// Creating modified containerToken
Token containerToken2 =
tamperedContainerTokenSecretManager.createContainerToken(cId2, nodeId,
user, r, Priority.newInstance(0), 0);
tamperedContainerTokenSecretManager.createContainerToken(cId2, 0,
nodeId, user, r, Priority.newInstance(0), 0);
StringBuilder sb = new StringBuilder("Given Container ");
sb.append(cId2);
@ -731,8 +731,8 @@ private void testContainerTokenWithEpoch(Configuration conf)
getContainerTokenSecretManager();
Resource r = Resource.newInstance(1230, 2);
Token containerToken =
containerTokenSecretManager.createContainerToken(cId, nodeId, user, r,
Priority.newInstance(0), 0);
containerTokenSecretManager.createContainerToken(cId, 0, nodeId, user,
r, Priority.newInstance(0), 0);
ContainerTokenIdentifier containerTokenIdentifier =
new ContainerTokenIdentifier();

View File

@ -198,8 +198,6 @@ private NodeStatus createNodeStatus(
/**
* Verify both the RMNode and SchedulerNode have been updated with the test
* fixture utilization data.
* @param containersUtilization Utilization of the container.
* @param nodeUtilization Utilization of the node.
*/
private void verifySimulatedUtilization() throws InterruptedException {
ResourceManager rm = cluster.getResourceManager(0);

View File

@ -44,5 +44,6 @@ message ContainerTokenIdentifierForTestProto {
optional int64 creationTime = 9;
optional LogAggregationContextProto logAggregationContext = 10;
optional string message = 11;
optional int32 version = 14 [default = 0];
}