It might worth filing a YARN jira to get it backported to 2.6.x and 2.7.x. At 
the very least, it will simplify rebuilding the timeline-server jar against the 
CDH version that you are running. 

— Hitesh

> On Aug 3, 2016, at 4:42 PM, Slava Markeyev <slava.marke...@upsight.com> wrote:
> 
> Thanks for the info Hitesh. Unfortunately it seems that RollingLevelDB is 
> only in trunk. I may have to backport it to 2.6.2 (version I use). I did 
> notice that the leveldb does grow to tens of gb which may be an indication of 
> pruning not happening often enough (or at all?). I also need to fix the 
> logging as the logs for the timeline server don't seem to be very active 
> beyond it starting up.
> 
> For the job I posted before here is the associated eventQueueBacklog log line.
> 2016-08-03 19:23:27,932 [INFO] [AMShutdownThread] 
> |ats.ATSHistoryLoggingService|: Stopping ATSService, eventQueueBacklog=17553
> I'll look into lowering tez.yarn.ats.event.flush.timeout.millis while trying 
> to look into the timelineserver.
> 
> Thanks for your help,
> Slava
> 
> On Wed, Aug 3, 2016 at 2:45 PM, Hitesh Shah <hit...@apache.org> wrote:
> Hello Slava,
> 
> Can you check for a log line along the lines of "Stopping ATSService, 
> eventQueueBacklog=“ to see how backed up is the event queue to YARN timeline?
> 
> I have noticed this in quite a few installs with YARN Timeline where YARN 
> Timeline is using the simple Level DB impl and not the RollingLevelDB storage 
> class. The YARN timeline ends up hitting some bottlenecks around the time 
> when the data purging happens ( takes a global lock on level db ). The 
> Rolling level db storage impl solved this problem by using separate level dos 
> for different time intervals and just throwing out the level db instead of 
> trying to do a full scan+purge.
> 
> Another workaround though not a great one is to set 
> “tez.yarn.ats.event.flush.timeout.millis” to a value say 60000 i.e. 1 min. 
> This implies that the Tez AM will try for at max 1 min to flush the queue to 
> YARN timeline before giving up and shutting down the Tez AM.
> 
> A longer term option is the YARN Timeline version 1.5 work currently slated 
> to be released in hadoop 2.8.0 which uses HDFS for writes instead of the 
> current web service based approach. This has a far better perf throughput for 
> writes albeit with a delay on the read path as the Timeline server scans HDFS 
> for new updates. The tez changes for this are already available in the source 
> code under the hadoop28 profile though the documentation for this is still 
> pending.
> 
> thanks
> — Hitesh
> 
> 
> 
> 
> 
> > On Aug 3, 2016, at 2:02 PM, Slava Markeyev <slava.marke...@upsight.com> 
> > wrote:
> >
> > I'm running into an issue that occurs fairly often (but not consistently 
> > reproducible) where yarn reports a negative value for memory allocation eg 
> > (-2048) and a 0 vcore allocation despite the AM actually running. For 
> > example the AM reports a runtime of 1hrs, 29mins, 40sec while the dag only 
> > 880 seconds.
> >
> > After some investigating I've noticed that the AM has repeated issues 
> > contacting the timeline server after the dag is complete (error trace 
> > below). This seems to be delaying the shutdown sequence. It seems to retry 
> > every minute before either giving up or succeeding but I'm not sure which. 
> > What's the best way to debug why this would be happening and potentially 
> > shortening the timeout retry period as I'm more concerned with job 
> > completion than logging it to the timeline server. This doesn't seem to be 
> > happening consistently to all tez jobs only some.
> >
> > I'm using hive 1.1.0 and tez 0.7.1 on cdh5.4.10 (hadoop 2.6).
> >
> > 2016-08-03 19:18:22,881 [INFO] [ContainerLauncher #112] 
> > |impl.ContainerManagementProtocolProxy|: Opening proxy : nodexxxx:45454
> > 2016-08-03 19:18:23,292 [WARN] [HistoryEventHandlingThread] 
> > |security.UserGroupInformation|: PriviledgedActionException as:xxxxx 
> > (auth:SIMPLE) cause:java.net.SocketTimeoutException: Read timed out
> > 2016-08-03 19:18:23,292 [ERROR] [HistoryEventHandlingThread] 
> > |impl.TimelineClientImpl|: Failed to get the response from the timeline 
> > server.
> > com.sun.jersey.api.client.ClientHandlerException: 
> > java.net.SocketTimeoutException: Read timed out
> >     at 
> > com.sun.jersey.client.urlconnection.URLConnectionClientHandler.handle(URLConnectionClientHandler.java:149)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineJerseyRetryFilter$1.run(TimelineClientImpl.java:226)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineClientConnectionRetry.retryOn(TimelineClientImpl.java:162)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineJerseyRetryFilter.handle(TimelineClientImpl.java:237)
> >     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:472)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl.doPosting(TimelineClientImpl.java:321)
> >     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:349)
> >     at 
> > org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService.access$700(ATSHistoryLoggingService.java:53)
> >     at 
> > org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService$1.run(ATSHistoryLoggingService.java:190)
> >     at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.net.SocketTimeoutException: Read timed out
> >     at java.net.SocketInputStream.socketRead0(Native Method)
> >     at java.net.SocketInputStream.read(SocketInputStream.java:152)
> >     at java.net.SocketInputStream.read(SocketInputStream.java:122)
> >     at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
> >     at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
> >     at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
> >     at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:689)
> >     at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:633)
> >     at 
> > sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1324)
> >     at 
> > java.net.HttpURLConnection.getResponseCode(HttpURLConnection.java:468)
> >     at 
> > org.apache.hadoop.security.authentication.client.AuthenticatedURL.extractToken(AuthenticatedURL.java:253)
> >     at 
> > org.apache.hadoop.security.authentication.client.PseudoAuthenticator.authenticate(PseudoAuthenticator.java:77)
> >     at 
> > org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator.authenticate(DelegationTokenAuthenticator.java:127)
> >     at 
> > org.apache.hadoop.security.authentication.client.AuthenticatedURL.openConnection(AuthenticatedURL.java:216)
> >     at 
> > org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.openConnection(DelegationTokenAuthenticatedURL.java:322)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineURLConnectionFactory$1.run(TimelineClientImpl.java:501)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineURLConnectionFactory$1.run(TimelineClientImpl.java:498)
> >     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:1707)
> >     at 
> > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineURLConnectionFactory.getHttpURLConnection(TimelineClientImpl.java:498)
> >     at 
> > com.sun.jersey.client.urlconnection.URLConnectionClientHandler._invoke(URLConnectionClientHandler.java:159)
> >     at 
> > com.sun.jersey.client.urlconnection.URLConnectionClientHandler.handle(URLConnectionClientHandler.java:147)
> >     ... 14 more
> >
> > and finally
> >
> > 2016-08-03 20:32:51,041 [INFO] [AMShutdownThread] 
> > |ats.ATSHistoryLoggingService|: Event queue empty, stopping ATS Service
> > 2016-08-03 20:32:51,131 [INFO] [AMShutdownThread] 
> > |launcher.ContainerLauncherImpl|: Stopping 
> > container_e12_1470097176422_30703_01_002211
> >
> >
> > Thanks,
> > Slava
> >
> > --
> > Slava Markeyev | Engineering | Upsight
> 
> 
> 
> 
> -- 
> Slava Markeyev | Engineering | Upsight

Reply via email to