You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-issues@hadoop.apache.org by "Bibin A Chundatt (JIRA)" <ji...@apache.org> on 2016/08/21 12:42:20 UTC

[jira] [Commented] (YARN-5537) Intermittent test failure of TestAMRMClient#testAMRMClientWithContainerResourceChange

    [ https://issues.apache.org/jira/browse/YARN-5537?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15429703#comment-15429703 ] 

Bibin A Chundatt commented on YARN-5537:
----------------------------------------

*Analysis*
AM container allocated to node {{localhost:43931}} - container1
container2,container3,container4 allocated to node {{localhost:36489}}

Increase container request gets served once heartbeat is received from {{localhost:36489}}.
During random failure node heartbeat request is received from node {{localhost:36489}} during allocate request and increase request is served.

{noformat}
2016-08-20 21:57:42,952 DEBUG [IPC Server handler 2 on 40133] ipc.Server: IPC Server handler 2 on 40133: org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB.allocate from 127.0.0.1:46570 Call#16 Retry#0 for RpcKind RPC_PROTOCOL_BUFFER
2016-08-20 21:57:42,952 DEBUG [IPC Server handler 2 on 40133] security.UserGroupInformation: PrivilegedAction as:appattempt_1471710454871_0001_000001 (auth:TOKEN) from:org.apache.hadoop.ipc.Server$Handler.run(Server.java:2419)
2016-08-20 21:57:42,956 DEBUG [IPC Server handler 2 on 40133] rmcontainer.RMContainerImpl: Processing container_1471710454871_0001_01_000004 of type RELEASED
2016-08-20 21:57:42,957 INFO  [IPC Server handler 2 on 40133] rmcontainer.RMContainerImpl: container_1471710454871_0001_01_000004 Container Transitioned from RUNNING to RELEASED
2016-08-20 21:57:42,958 INFO  [IPC Server handler 2 on 40133] resourcemanager.RMAuditLogger: USER=root	IP=127.0.0.1	OPERATION=AM Released Container	TARGET=SchedulerApp	RESULT=SUCCESS	APPID=application_1471710454871_0001	CONTAINERID=container_1471710454871_0001_01_000004	RESOURCE=<memory:1024, vCores:1>
2016-08-20 21:57:42,958 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent.EventType: STATUS_UPDATE
2016-08-20 21:57:42,958 DEBUG [AsyncDispatcher event handler] attempt.RMAppAttemptImpl: Processing event for appattempt_1471710454871_0001_000001 of type STATUS_UPDATE
2016-08-20 21:57:42,958 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent.EventType: CLEANUP_CONTAINER
2016-08-20 21:57:42,958 DEBUG [AsyncDispatcher event handler] rmnode.RMNodeImpl: Processing localhost:46667 of type CLEANUP_CONTAINER
2016-08-20 21:57:42,958 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent.EventType: CONTAINER_FINISHED
2016-08-20 21:57:42,958 DEBUG [AsyncDispatcher event handler] attempt.RMAppAttemptImpl: Processing event for appattempt_1471710454871_0001_000001 of type CONTAINER_FINISHED
2016-08-20 21:57:42,958 DEBUG [IPC Server handler 2 on 40133] scheduler.SchedulerNode: Released container container_1471710454871_0001_01_000004 of capacity <memory:1024, vCores:1> on host localhost:46667, which currently has 2 containers, <memory:2048, vCores:2> used and <memory:2048, vCores:6> available, release resources=true
2016-08-20 21:57:42,963 DEBUG [IPC Server handler 2 on 40133] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:3072, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:3072, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.25 Partition: 
2016-08-20 21:57:42,963 DEBUG [IPC Server handler 2 on 40133] capacity.LeafQueue: default used=<memory:3072, vCores:3> numContainers=3 user=root user-resources=<memory:3072, vCores:3>
2016-08-20 21:57:42,963 DEBUG [IPC Server handler 2 on 40133] capacity.ParentQueue: completedContainer root: numChildQueue= 1, capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:3072, vCores:3>usedCapacity=0.25, numApps=1, numContainers=3, cluster=<memory:12288, vCores:24>
2016-08-20 21:57:42,963 DEBUG [IPC Server handler 2 on 40133] capacity.ParentQueue: Re-sorting completed queue: default: capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:3072, vCores:3>, usedCapacity=0.25, absoluteUsedCapacity=0.25, numApps=1, numContainers=3
2016-08-20 21:57:42,967 WARN  [IPC Server handler 2 on 40133] scheduler.AbstractYarnScheduler: Error happens when checking increase request, Ignoring.. exception=
org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException: Failed to get rmContainer for increase request, with container-id=container_1471710454871_0001_01_000004
	at org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler.createSchedContainerChangeRequest(AbstractYarnScheduler.java:768)
	at org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler.createSchedContainerChangeRequests(AbstractYarnScheduler.java:785)
	at org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler.updateIncreaseRequests(CapacityScheduler.java:934)
	at org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler.allocate(CapacityScheduler.java:968)
	at org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService.allocate(ApplicationMasterService.java:525)
	at org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl.allocate(ApplicationMasterProtocolPBServiceImpl.java:60)
	at org.apache.hadoop.yarn.proto.ApplicationMasterProtocol$ApplicationMasterProtocolService$2.callBlockingMethod(ApplicationMasterProtocol.java:99)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:663)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:989)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2423)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2419)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1790)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2419)
2016-08-20 21:57:42,973 DEBUG [IPC Server handler 2 on 40133] scheduler.AppSchedulingInfo: Added increase request:container_1471710454871_0001_01_000003 delta=<memory:1024, vCores:0>
2016-08-20 21:57:42,974 DEBUG [IPC Server handler 2 on 40133] scheduler.AbstractYarnScheduler: Processing decrease request:<container=container_1471710454871_0001_01_000002, targetCapacity=<memory:512, vCores:1>, node=localhost:46667>
2016-08-20 21:57:42,975 DEBUG [IPC Server handler 2 on 40133] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,975 DEBUG [IPC Server handler 2 on 40133] capacity.LeafQueue: default used=<memory:2560, vCores:3> numContainers=3 user=root user-resources=<memory:2560, vCores:3>
2016-08-20 21:57:42,975 DEBUG [IPC Server handler 2 on 40133] scheduler.AppSchedulingInfo: Decrease container : applicationId=application_1471710454871_0001 container=container_1471710454871_0001_01_000002 host=localhost:46667 user=root resource=<memory:512, vCores:0>
2016-08-20 21:57:42,976 DEBUG [IPC Server handler 2 on 40133] rmcontainer.RMContainerImpl: Processing container_1471710454871_0001_01_000002 of type CHANGE_RESOURCE
2016-08-20 21:57:42,977 DEBUG [IPC Server handler 2 on 40133] scheduler.SchedulerNode: Decreased container container_1471710454871_0001_01_000002 of capacity <memory:512, vCores:0> on host localhost:46667, which has 2 containers, <memory:1536, vCores:2> used and <memory:2560, vCores:6> available after allocation
2016-08-20 21:57:42,977 DEBUG [IPC Server handler 2 on 40133] capacity.ParentQueue: completedContainer root: numChildQueue= 1, capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:2560, vCores:3>usedCapacity=0.20833333, numApps=1, numContainers=2, cluster=<memory:12288, vCores:24>
2016-08-20 21:57:42,978 INFO  [IPC Server handler 2 on 40133] capacity.LeafQueue: Application attempt appattempt_1471710454871_0001_000001 decreased container:container_1471710454871_0001_01_000002 from <memory:1024, vCores:1> to <memory:512, vCores:1>
2016-08-20 21:57:42,979 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent.EventType: DECREASE_CONTAINER
2016-08-20 21:57:42,981 DEBUG [AsyncDispatcher event handler] rmnode.RMNodeImpl: Processing localhost:46667 of type DECREASE_CONTAINER
2016-08-20 21:57:42,982 DEBUG [IPC Server handler 2 on 40133] security.BaseContainerTokenSecretManager: Creating password for container_1471710454871_0001_01_000002 for user container_1471710454871_0001_01_000002 (auth:SIMPLE) to be run on NM localhost:46667
2016-08-20 21:57:42,982 DEBUG [IPC Server handler 2 on 40133] security.ContainerTokenIdentifier: Writing ContainerTokenIdentifier to RPC layer: containerId { app_attempt_id { application_id { id: 1 cluster_timestamp: 1471710454871 } attemptId: 1 } id: 2 } nmHostAddr: "localhost:46667" appSubmitter: "root" resource { memory: 512 virtual_cores: 1 } expiryTimeStamp: 1471711062979 masterKeyId: 525038130 rmIdentifier: 1471710454871 priority { priority: 1 } creationTime: 1471710462201 nodeLabelExpression: "" containerType: TASK executionType: GUARANTEED
2016-08-20 21:57:42,983 DEBUG [IPC Server handler 2 on 40133] security.ContainerTokenIdentifier: Writing ContainerTokenIdentifier to RPC layer: containerId { app_attempt_id { application_id { id: 1 cluster_timestamp: 1471710454871 } attemptId: 1 } id: 2 } nmHostAddr: "localhost:46667" appSubmitter: "root" resource { memory: 512 virtual_cores: 1 } expiryTimeStamp: 1471711062979 masterKeyId: 525038130 rmIdentifier: 1471710454871 priority { priority: 1 } creationTime: 1471710462201 nodeLabelExpression: "" containerType: TASK executionType: GUARANTEED
2016-08-20 21:57:42,985 DEBUG [IPC Server handler 2 on 40133] rmcontainer.RMContainerImpl: Processing container_1471710454871_0001_01_000002 of type ACQUIRE_UPDATED_CONTAINER
2016-08-20 21:57:42,985 DEBUG [IPC Server handler 2 on 40133] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,986 DEBUG [Node Status Updater] nodemanager.NodeStatusUpdaterImpl: Node's health-status : true, 
2016-08-20 21:57:42,989 DEBUG [Node Status Updater] nodemanager.NodeStatusUpdaterImpl: Sending out 0 container statuses: []
2016-08-20 21:57:42,990 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent.EventType: STATUS_UPDATE
2016-08-20 21:57:42,991 DEBUG [AsyncDispatcher event handler] rmnode.RMNodeImpl: Processing localhost:43381 of type STATUS_UPDATE
2016-08-20 21:57:42,991 DEBUG [IPC Server handler 2 on 40133] ipc.Server: Served: allocate queueTime= 1 procesingTime= 38
2016-08-20 21:57:42,991 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent.EventType: NODE_UPDATE
2016-08-20 21:57:42,991 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.CapacityScheduler: nodeUpdate: localhost:43381 clusterResources: <memory:12288, vCores:24>
2016-08-20 21:57:42,991 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.CapacityScheduler: Node being looked for scheduling localhost:43381 availableResource: <memory:4096, vCores:8>
2016-08-20 21:57:42,992 DEBUG [IPC Server handler 2 on 40133] ipc.Server: IPC Server handler 2 on 40133: responding to org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB.allocate from 127.0.0.1:46570 Call#16 Retry#0
2016-08-20 21:57:42,991 DEBUG [Node Status Updater] nodemanager.NodeStatusUpdaterImpl: Node's health-status : true, 
2016-08-20 21:57:42,992 DEBUG [IPC Client (1201360998) connection to localhost/127.0.0.1:40133 from root] ipc.Client: IPC Client (1201360998) connection to localhost/127.0.0.1:40133 from root got value #16
2016-08-20 21:57:42,992 DEBUG [Thread-346] ipc.ProtobufRpcEngine: Call: allocate took 41ms
2016-08-20 21:57:42,992 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.CapacityScheduler: Trying to schedule on node: localhost, available: <memory:4096, vCores:8>
2016-08-20 21:57:42,993 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.ParentQueue: Trying to assign containers to child-queue of root
2016-08-20 21:57:42,993 DEBUG [Thread-346] impl.AMRMClientImpl: RM has confirmed changed resource allocation for container container_1471710454871_0001_01_000002. Current resource allocation:<memory:512, vCores:1>. Remove pending change request:<memory:512, vCores:1>
2016-08-20 21:57:42,993 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.ParentQueue: printChildQueues - queue: root child-queues: root.defaultusedCapacity=(0.20833333),  label=(*)
2016-08-20 21:57:42,993 DEBUG [Node Status Updater] nodemanager.NodeStatusUpdaterImpl: Sending out 3 container statuses: [ContainerStatus: [ContainerId: container_1471710454871_0001_01_000002, ExecutionType: GUARANTEED, State: RUNNING, Capability: <memory:1024, vCores:1>, Diagnostics: , ExitStatus: -1000, ], ContainerStatus: [ContainerId: container_1471710454871_0001_01_000003, ExecutionType: GUARANTEED, State: RUNNING, Capability: <memory:1024, vCores:1>, Diagnostics: , ExitStatus: -1000, ], ContainerStatus: [ContainerId: container_1471710454871_0001_01_000004, ExecutionType: GUARANTEED, State: RUNNING, Capability: <memory:1024, vCores:1>, Diagnostics: , ExitStatus: -1000, ]]
2016-08-20 21:57:42,993 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.ParentQueue: Trying to assign to queue: root.default stats: default: capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:2560, vCores:3>, usedCapacity=0.20833333, absoluteUsedCapacity=0.20833333, numApps=1, numContainers=3
2016-08-20 21:57:42,993 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: assignContainers: node=localhost #applications=1
2016-08-20 21:57:42,993 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: Headroom calculation for user root:  userLimit=<memory:12288, vCores:1> queueMaxAvailRes=<memory:12288, vCores:1> consumed=<memory:2560, vCores:3> headroom=<memory:9728, vCores:21>
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] fica.FiCaSchedulerApp: pre-assignContainers for application application_1471710454871_0001
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 headRoom=<memory:9728, vCores:21> currentConsumption=2560
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 request={AllocationRequestId: 0, Priority: 0, Capability: <memory:1024, vCores:1>, # Containers: 0, Location: *, Relax Locality: true, Execution Type Request: {Execution Type: GUARANTEED, Enforce Execution Type: false}, Node Label Expression: }
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 headRoom=<memory:9728, vCores:21> currentConsumption=2560
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 request={AllocationRequestId: 0, Priority: 1, Capability: <memory:1024, vCores:1>, # Containers: 0, Location: *, Relax Locality: true, Execution Type Request: {Execution Type: GUARANTEED, Enforce Execution Type: false}, Node Label Expression: }
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] allocator.IncreaseContainerAllocator: Skip allocating increase request since we don't have any increase request on this node=localhost:43381
2016-08-20 21:57:42,994 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: post-assignContainers for application application_1471710454871_0001
2016-08-20 21:57:42,995 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,995 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 headRoom=<memory:9728, vCores:21> currentConsumption=2560
2016-08-20 21:57:42,995 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 request={AllocationRequestId: 0, Priority: 0, Capability: <memory:1024, vCores:1>, # Containers: 0, Location: *, Relax Locality: true, Execution Type Request: {Execution Type: GUARANTEED, Enforce Execution Type: false}, Node Label Expression: }
2016-08-20 21:57:42,995 DEBUG [SchedulerEventDispatcher:Event Processor] capacity.LeafQueue: User limit computation for root in queue default userLimitPercent=100 userLimitFactor=1.0 required: <memory:512, vCores:1> consumed: <memory:2560, vCores:1> user-limit-resource: <memory:12288, vCores:1> queueCapacity: <memory:12288, vCores:1> qconsumed: <memory:2560, vCores:3> consumedRatio: 0.0 currentCapacity: <memory:12288, vCores:1> activeUsers: 0 clusterCapacity: <memory:12288, vCores:24> resourceByLabel: <memory:12288, vCores:24> usageratio: 0.20833333 Partition: 
2016-08-20 21:57:42,995 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 headRoom=<memory:9728, vCores:21> currentConsumption=2560
2016-08-20 21:57:42,995 DEBUG [SchedulerEventDispatcher:Event Processor] scheduler.SchedulerApplicationAttempt: showRequests: application=application_1471710454871_0001 request={AllocationRequestId: 0, Priority: 1, Capability: <memory:1024, vCores:1>, # Containers: 0, Location: *, Relax Locality: true, Execution Type Request: {Execution Type: GUARANTEED, Enforce Execution Type: false}, Node Label Expression: }
2016-08-20 21:57:42,995 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent.EventType: FINISH_CONTAINERS
2016-08-20 21:57:43,002 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent.EventType: DECREASE_CONTAINERS_RESOURCE
2016-08-20 21:57:42,996 DEBUG [AsyncDispatcher event handler] event.AsyncDispatcher: Dispatching the event org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent.EventType: STATUS_UPDATE
2016-08-20 21:57:43,005 DEBUG [AsyncDispatcher event handler] rmnode.RMNodeImpl: Processing localhost:46667 of type STATUS_UPDATE
2016-08-20 21:57:42,995 DEBUG [IPC Server handler 2 on 40133] ipc.Server: IPC Server handler 2 on 40133: responding to org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB.allocate from 127.0.0.1:46570 Call#16 Retry#0 Wrote 334 bytes.
{noformat}

> Intermittent test failure of TestAMRMClient#testAMRMClientWithContainerResourceChange
> -------------------------------------------------------------------------------------
>
>                 Key: YARN-5537
>                 URL: https://issues.apache.org/jira/browse/YARN-5537
>             Project: Hadoop YARN
>          Issue Type: Bug
>            Reporter: Varun Saxena
>            Assignee: Bibin A Chundatt
>         Attachments: Failure.txt
>
>
> Refer to test report https://builds.apache.org/job/PreCommit-YARN-Build/12692/testReport/
> {noformat}
> Running org.apache.hadoop.yarn.client.api.impl.TestAMRMClient
> Tests run: 1, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 12.018 sec <<< FAILURE! - in org.apache.hadoop.yarn.client.api.impl.TestAMRMClient
> testAMRMClientWithContainerResourceChange(org.apache.hadoop.yarn.client.api.impl.TestAMRMClient)  Time elapsed: 1.183 sec  <<< FAILURE!
> java.lang.AssertionError: expected:<0> but was:<1>
> 	at org.junit.Assert.fail(Assert.java:88)
> 	at org.junit.Assert.failNotEquals(Assert.java:743)
> 	at org.junit.Assert.assertEquals(Assert.java:118)
> 	at org.junit.Assert.assertEquals(Assert.java:555)
> 	at org.junit.Assert.assertEquals(Assert.java:542)
> 	at org.apache.hadoop.yarn.client.api.impl.TestAMRMClient.doContainerResourceChange(TestAMRMClient.java:1019)
> 	at org.apache.hadoop.yarn.client.api.impl.TestAMRMClient.testAMRMClientWithContainerResourceChange(TestAMRMClient.java:909)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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