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

Rajesh Balamohan commented on YARN-3942:
----------------------------------------

Should this be resilient to cluster restarts? For e.g, when cluster restart 
happens, timeline server automatically gets killed with the following exception.

{noformat}
2015-08-18 01:03:31,523 [EntityLogPluginWorker #6] ERROR 
org.apache.hadoop.yarn.server.timeline.EntityFileTimelineStore: Error scanning 
active files
...
...
[EntityLogPluginWorker #0] ERROR 
org.apache.hadoop.yarn.server.timeline.EntityFileTimelineStore: Error scanning 
active files
java.io.EOFException: End of File Exception between local host is: 
"atsmachine"; destination host is: "m1":8020; : java.io.EOFException; For more 
details see:  http://wiki.apache.org/hadoop/EOFException
        at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
        at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:422)
        at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:792)
        at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:765)
        at org.apache.hadoop.ipc.Client.call(Client.java:1444)
        at org.apache.hadoop.ipc.Client.call(Client.java:1371)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
        at com.sun.proxy.$Proxy26.getListing(Unknown Source)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getListing(ClientNamenodeProtocolTranslatorPB.java:574)
        at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:252)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
        at com.sun.proxy.$Proxy27.getListing(Unknown Source)
        at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1748)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:973)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:984)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:956)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$21.doCall(DistributedFileSystem.java:935)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$21.doCall(DistributedFileSystem.java:931)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.listStatusIterator(DistributedFileSystem.java:943)
        at 
org.apache.hadoop.yarn.server.timeline.EntityFileTimelineStore.scanActiveLogs(EntityFileTimelineStore.java:314)
        at 
org.apache.hadoop.yarn.server.timeline.EntityFileTimelineStore.access$1300(EntityFileTimelineStore.java:79)
        at 
org.apache.hadoop.yarn.server.timeline.EntityFileTimelineStore$EntityLogScanner.run(EntityFileTimelineStore.java:771)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.EOFException
        at java.io.DataInputStream.readInt(DataInputStream.java:392)
        at 
org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1098)
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:993)
2015-08-18 01:03:35,600 [SIGTERM handler] ERROR 
org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer:
 RECEIVED SIGNAL 15: SIGTERM
2015-08-18 01:03:35,608 [Thread-1] INFO org.mortbay.log: Stopped 
HttpServer2$SelectChannelConnectorWithSafeStartup@atsmachine:8188
2015-08-18 01:03:35,710 [Thread-1] INFO 
org.apache.hadoop.metrics2.impl.MetricsSystemImpl: Stopping 
ApplicationHistoryServer metrics system...
2015-08-18 01:03:35,712 [Thread-1] INFO 
org.apache.hadoop.metrics2.impl.MetricsSystemImpl: ApplicationHistoryServer 
metrics system stopped.
2015-08-18 01:03:35,713 [Thread-1] INFO 
org.apache.hadoop.metrics2.impl.MetricsSystemImpl: ApplicationHistoryServer 
metrics system shutdown complete.
{noformat}

> Timeline store to read events from HDFS
> ---------------------------------------
>
>                 Key: YARN-3942
>                 URL: https://issues.apache.org/jira/browse/YARN-3942
>             Project: Hadoop YARN
>          Issue Type: Improvement
>          Components: timelineserver
>            Reporter: Jason Lowe
>            Assignee: Jason Lowe
>         Attachments: YARN-3942.001.patch
>
>
> This adds a new timeline store plugin that is intended as a stop-gap measure 
> to mitigate some of the issues we've seen with ATS v1 while waiting for ATS 
> v2.  The intent of this plugin is to provide a workable solution for running 
> the Tez UI against the timeline server on a large-scale clusters running many 
> thousands of jobs per day.



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

Reply via email to