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

Rohini Palaniswamy commented on TEZ-2300:
-----------------------------------------

This one was a case of AM lingering long due to posting to ATS. [~jlowe], did 
configure the tez.yarn.ats.max.events.per.batch to 500 in tez-site.xml  to make 
the problem better. 

2015-04-12 23:24:42,040 [Timer-0] INFO  
org.apache.pig.backend.hadoop.executionengine.tez.TezJob - DAG Status: 
status=RUNNING, progress=TotalTasks: 51890 Succeeded: 0 Running: 50 Failed: 0 
Killed: 0, diagnostics=, counters=null

Did Ctlr+C here on the pig client.

AM DAG log:
2015-04-12 23:24:52,015 INFO [AsyncDispatcher event handler] app.DAGAppMaster: 
DAG completed, dagId=dag_1428329756093_325099_1, dagState=KILLED
2015-04-12 23:24:52,015 INFO [AsyncDispatcher event handler] 
common.TezUtilsInternal: Redirecting log file based on addend: 
dag_1428329756093_325099_1_post

In the dag_1428329756093_325099_1_post log (attached to the jira as well)

{code}
2015-04-12 23:24:57,029 INFO [AMShutdownThread] ats.ATSHistoryLoggingService: 
Stopping ATSService, eventQueueBacklog=17927
2015-04-12 23:25:25,466 WARN [AMShutdownThread] ats.ATSHistoryLoggingService: 
ATSService being stopped, eventQueueBacklog=17927, maxTimeLeftToFlush=-1, 
waitForever=true
.... Lot of ATS put errors ....
2015-04-12 23:32:53,197 INFO [AMShutdownThread] ats.ATSHistoryLoggingService: 
Event queue empty, stopping ATS Service
2015-04-12 23:32:53,200 INFO [DelayedContainerManager] 
rm.YarnTaskSchedulerService: AllocatedContainerManager Thread interrupted
2015-04-12 23:32:53,203 INFO [AMShutdownThread] rm.YarnTaskSchedulerService: 
Unregistering application from RM, exitStatus=SUCCEEDED, exitMessage=Session 
stats:submittedDAGs=1, successfulDAGs=0, failedDAGs=0, killedDAGs=1
, 
trackingURL=bassniumtan-jt1.tan.ygrid.yahoo.com:4080/tez/#/?appid=application_1428329756093_325099
2015-04-12 23:32:53,210 INFO [AMShutdownThread] impl.AMRMClientImpl: Waiting 
for application to be successfully unregistered.
2015-04-12 23:32:53,314 INFO [AMShutdownThread] rm.YarnTaskSchedulerService: 
Successfully unregistered application from RM
2015-04-12 23:32:53,315 INFO [AMRM Callback Handler Thread] 
impl.AMRMClientAsyncImpl: Interrupted while waiting for queue
java.lang.InterruptedException
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2017)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2052)
        at 
java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at 
org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl$CallbackHandlerThread.run(AMRMClientAsyncImpl.java:274)
2015-04-12 23:32:53,316 INFO [AMShutdownThread] ipc.Server: Stopping server on 
51921
2015-04-12 23:32:53,319 INFO [IPC Server listener on 51921] ipc.Server: 
Stopping IPC Server listener on 51921
2015-04-12 23:32:53,319 INFO [AMShutdownThread] ipc.Server: Stopping server on 
50500
2015-04-12 23:32:53,320 INFO [IPC Server listener on 50500] ipc.Server: 
Stopping IPC Server listener on 50500
2015-04-12 23:32:53,320 INFO [IPC Server Responder] ipc.Server: Stopping IPC 
Server Responder
2015-04-12 23:32:53,320 INFO [IPC Server Responder] ipc.Server: Stopping IPC 
Server Responder
2015-04-12 23:32:53,324 INFO [AMShutdownThread] app.DAGAppMaster: Completed 
deletion of tez scratch data dir, 
path=hdfs://bassniumtan-nn1.tan.ygrid.yahoo.com:8020/tmp/temp-1464028011/.tez/application_1428329756093_325099
2015-04-12 23:32:53,324 INFO [AMShutdownThread] app.DAGAppMaster: Exiting 
DAGAppMaster..GoodBye!
2015-04-12 23:32:53,325 INFO [Thread-1] app.DAGAppMaster: 
DAGAppMasterShutdownHook invoked
{code}

Jstack still running thread on AM :
{code}
"AMShutdownThread" prio=10 tid=0x0f3ad800 nid=0x2b0e runnable [0x04e40000]
   java.lang.Thread.State: RUNNABLE
        at java.net.SocketInputStream.socketRead0(Native Method)
        at java.net.SocketInputStream.read(SocketInputStream.java:150)
        at java.net.SocketInputStream.read(SocketInputStream.java:121)
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
        at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
        - locked <0xde56b700> (a java.io.BufferedInputStream)
        at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:633)
        at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:579)
        at 
sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1322)
        - locked <0xde42d320> (a sun.net.www.protocol.http.HttpURLConnection)
        at 
java.net.HttpURLConnection.getResponseCode(HttpURLConnection.java:468)
        at 
com.sun.jersey.client.urlconnection.URLConnectionClientHandler._invoke(URLConnectionClientHandler.java:240)
        at 
com.sun.jersey.client.urlconnection.URLConnectionClientHandler.handle(URLConnectionClientHandler.java:147)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineJerseyRetryFilter$1.run(TimelineClientImpl.java:230)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineClientConnectionRetry.retryOn(TimelineClientImpl.java:166)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineJerseyRetryFilter.handle(TimelineClientImpl.java:241)
        at com.sun.jersey.api.client.Client.handle(Client.java:648)
        at com.sun.jersey.api.client.WebResource.handle(WebResource.java:670)
        at com.sun.jersey.api.client.WebResource.access$200(WebResource.java:74)
        at 
com.sun.jersey.api.client.WebResource$Builder.post(WebResource.java:563)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl.doPostingObject(TimelineClientImpl.java:461)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$1.run(TimelineClientImpl.java:319)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$1.run(TimelineClientImpl.java:316)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:415)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1694)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl.doPosting(TimelineClientImpl.java:316)
        at 
org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl.putEntities(TimelineClientImpl.java:301)
        at 
org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService.handleEvents(ATSHistoryLoggingService.java:304)
        at 
org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService.serviceStop(ATSHistoryLoggingService.java:199)
        - locked <0x1481e8a0> (a java.lang.Object)
        at 
org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
        - locked <0x1481e7d0> (a java.lang.Object)
        at 
org.apache.hadoop.service.ServiceOperations.stop(ServiceOperations.java:52)
        at 
org.apache.hadoop.service.ServiceOperations.stopQuietly(ServiceOperations.java:80)
        at 
org.apache.hadoop.service.CompositeService.stop(CompositeService.java:157)
        at 
org.apache.hadoop.service.CompositeService.serviceStop(CompositeService.java:131)
        at 
org.apache.tez.dag.history.HistoryEventHandler.serviceStop(HistoryEventHandler.java:80)
        at 
org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
        - locked <0x147fd580> (a java.lang.Object)
        at 
org.apache.hadoop.service.ServiceOperations.stop(ServiceOperations.java:52)
        at 
org.apache.hadoop.service.ServiceOperations.stopQuietly(ServiceOperations.java:80)
        at 
org.apache.tez.dag.app.DAGAppMaster.stopServices(DAGAppMaster.java:1535)
        at 
org.apache.tez.dag.app.DAGAppMaster.serviceStop(DAGAppMaster.java:1674)
        - locked <0x145e1238> (a org.apache.tez.dag.app.DAGAppMaster)
        at 
org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
        - locked <0x145e1378> (a java.lang.Object)
        at 
org.apache.tez.dag.app.DAGAppMaster$DAGAppMasterShutdownHandler$AMShutdownRunnable.run(DAGAppMaster.java:724)
        at java.lang.Thread.run(Thread.java:722)
{code}

ATS UI shows start and end time as 12 Apr 2015 16:24:20 and 12 Apr 2015 
16:24:51 for the DAG and status as KILLED.  But the AM started at 12-Apr-2015 
23:24:05 and only finished after 8mins, 47sec. Not sure why there are so many 
ATS put errors. It is those retries that are causing so much time.

> TezClient.stop() takes a lot of time or does not work sometimes
> ---------------------------------------------------------------
>
>                 Key: TEZ-2300
>                 URL: https://issues.apache.org/jira/browse/TEZ-2300
>             Project: Apache Tez
>          Issue Type: Bug
>            Reporter: Rohini Palaniswamy
>
>   Noticed this with a couple of pig scripts which were not behaving well (AM 
> close to OOM, etc) and even with some that were running fine. Pig calls 
> Tezclient.stop() in shutdown hook. Ctrl+C to the pig script either exits 
> immediately or is hung. In both cases it either takes a long time for the 
> yarn application to go to KILLED state. Many times I just end up calling yarn 
> application -kill separately after waiting for 5 mins or more for it to get 
> killed.



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

Reply via email to