You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Saisai Shao <sa...@gmail.com> on 2015/08/18 11:23:02 UTC

Re: How should I do to solve this problem that the executorsofmysparkapplication always is blocked after an executor is lost?

I'm not sure it is caused by memory issue, from the log you pasted, seems
there's no very useful information could conclude that this issue is caused
by memory overflow.

Could you please share your memory size of executor (spark.executor.memory)
and number of executors run on each node.

Another related issue is that if you use too many memory that is not enough
for kernel to work, OS will kill the process silently. But from your log I
don't think it is caused by this.

I guess container_1439803298368_0005_01_000001 is application master, its
exit code 15 means EXIT_EXCEPTION_USER_CLASS. I'm not sure what actual
exception is and what could lead to such exception?

I think you'd better do some more tests to reproduce the issue and identify
the problem, sorry currently I've no idea what is the actual cause.

Thanks
Saisai


On Tue, Aug 18, 2015 at 3:18 PM, 刚 <94...@qq.com> wrote:

> Hi Saisai:
>     I find the file as follows:
>     /export/servers/hadoop2.6.0/logs/
> yarn-root-nodemanager-A01-R08-2-I160-103.JD.LOCAL.log
>     I think it should be the node manager log.
>     I find some interesting things in this file.The logs are as follows.
> 2015-08-17 17:25:41,550 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl:
> Start request for container_1439803298368_0005_01_000001 by user root
> 2015-08-17 17:25:41,551 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl:
> Creating a new application reference for app application_1439803298368_0005
> 2015-08-17 17:25:41,551 INFO
> org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger: USER=root
> IP=172.19.160.102 OPERATION=Start Container Request
> TARGET=ContainerManageImpl RESULT=SUCCESS
> APPID=application_1439803298368_0005
> CONTAINERID=container_1439803298368_0005_01_000001
> 2015-08-17 17:25:41,551 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application:
> Application application_1439803298368_0005 transitioned from NEW to INITING
> 2015-08-17 17:25:41,552 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application:
> Adding container_1439803298368_0005_01_000001 to application
> application_1439803298368_0005
> 2015-08-17 17:25:41,557 WARN
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AppLogAggregatorImpl:
> rollingMonitorInterval is set as -1. The log rolling mornitoring interval
> is disabled. The logs will be aggregated after this application is finished.
> 2015-08-17 17:25:41,663 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application:
> Application application_1439803298368_0005 transitioned from INITING to
> RUNNING
> 2015-08-17 17:25:41,664 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000001 transitioned from NEW to
> LOCALIZING
> 2015-08-17 17:25:41,664 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices: Got
> event CONTAINER_INIT for appId application_1439803298368_0005
> 2015-08-17 17:25:41,664 INFO
> org.apache.spark.network.yarn.YarnShuffleService: Initializing container
> container_1439803298368_0005_01_000001
> 2015-08-17 17:25:41,665 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource:
> Resource
> hdfs://A01-R08-3-I160-102.JD.LOCAL:9000/user/root/.sparkStaging/application_1439803298368_0005/spark-assembly-1.3.1-hadoop2.6.0.jar
> transitioned from INIT to DOWNLOADING
> 2015-08-17 17:25:41,665 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource:
> Resource
> hdfs://A01-R08-3-I160-102.JD.LOCAL:9000/user/root/.sparkStaging/application_1439803298368_0005/spark_Security-1.0-SNAPSHOT.jar
> transitioned from INIT to DOWNLOADING
> 2015-08-17 17:25:41,665 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService:
> Created localizer for container_1439803298368_0005_01_000001
> 2015-08-17 17:25:41,668 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService:
> Writing credentials to the nmPrivate file
> /export/servers/hadoop2.6.0/tmp/nm-local-dir/nmPrivate/container_1439803298368_0005_01_000001.tokens.
> Credentials list:
> 2015-08-17 17:25:41,682 INFO
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor:
> Initializing user root
> 2015-08-17 17:25:41,686 INFO
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor: Copying
> from
> /export/servers/hadoop2.6.0/tmp/nm-local-dir/nmPrivate/container_1439803298368_0005_01_000001.tokens
> to
> /export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/appcache/application_1439803298368_0005/container_1439803298368_0005_01_000001.tokens
> 2015-08-17 17:25:41,686 INFO
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor:
> Localizer CWD set to
> /export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/appcache/application_1439803298368_0005
> =
> file:/export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/appcache/application_1439803298368_0005
> 2015-08-17 17:25:42,240 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource:
> Resource
> hdfs://A01-R08-3-I160-102.JD.LOCAL:9000/user/root/.sparkStaging/application_1439803298368_0005/spark-assembly-1.3.1-hadoop2.6.0.jar(->/export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/filecache/14/spark-assembly-1.3.1-hadoop2.6.0.jar)
> transitioned from DOWNLOADING to LOCALIZED
> 2015-08-17 17:25:42,508 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource:
> Resource
> hdfs://A01-R08-3-I160-102.JD.LOCAL:9000/user/root/.sparkStaging/application_1439803298368_0005/spark_Security-1.0-SNAPSHOT.jar(->/export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/filecache/15/spark_Security-1.0-SNAPSHOT.jar)
> transitioned from DOWNLOADING to LOCALIZED
> 2015-08-17 17:25:42,508 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000001 transitioned from
> LOCALIZING to LOCALIZED
> *2015-08-17 17:25:42,548 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000001 transitioned from
> LOCALIZED to RUNNING*
> ................................................
> 2015-08-17 17:26:20,366 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl:
> Start request for container_1439803298368_0005_01_000003 by user root
> 2015-08-17 17:26:20,367 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application:
> Adding container_1439803298368_0005_01_000003 to application
> application_1439803298368_0005
> 2015-08-17 17:26:20,368 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000003 transitioned from NEW to
> LOCALIZING
> 2015-08-17 17:26:20,368 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices: Got
> event CONTAINER_INIT for appId application_1439803298368_0005
> 2015-08-17 17:26:20,368 INFO
> org.apache.spark.network.yarn.YarnShuffleService: Initializing container
> container_1439803298368_0005_01_000003
> 2015-08-17 17:26:20,369 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000003 transitioned from
> LOCALIZING to LOCALIZED
> 2015-08-17 17:26:20,370 INFO
> org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger: USER=root
> IP=172.19.160.102 OPERATION=Start Container Request
> TARGET=ContainerManageImpl RESULT=SUCCESS
> APPID=application_1439803298368_0005
> CONTAINERID=container_1439803298368_0005_01_000003
> 2015-08-17 17:26:20,443 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000003 transitioned from
> LOCALIZED to RUNNING
> 2015-08-17 17:26:20,443 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl:
> Neither virutal-memory nor physical-memory monitoring is needed. Not
> running the monitor-thread
> 2015-08-17 17:26:20,449 INFO
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor:
> launchContainer: [bash,
> /export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/appcache/application_1439803298368_0005/container_1439803298368_0005_01_000003/default_container_executor.sh]
> ..........................................
>
> *2015-08-18 01:50:30,297 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch:
> Container container_1439803298368_0005_01_000003 succeeded *
> *2015-08-18 01:50:30,440 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000003 transitioned from RUNNING
> to EXITED_WITH_SUCCESS*
> 2015-08-18 01:50:30,465 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch:
> Cleaning up container container_1439803298368_0005_01_000003
> 2015-08-18 01:50:35,046 INFO
> org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger: USER=root OPERATION=Container
> Finished - Succeeded TARGET=ContainerImpl RESULT=SUCCESS
> APPID=application_1439803298368_0005
> CONTAINERID=container_1439803298368_0005_01_000003
> 2015-08-18 01:50:35,062 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container:
> Container container_1439803298368_0005_01_000003 transitioned from
> EXITED_WITH_SUCCESS to DONE
> 2015-08-18 01:50:35,065 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application:
> Removing container_1439803298368_0005_01_000003 from application
> application_1439803298368_0005
> 2015-08-18 01:50:35,070 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl:
> Neither virutal-memory nor physical-memory monitoring is needed. Not
> running the monitor-thread
> 2015-08-18 01:50:35,082 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AppLogAggregatorImpl:
> Considering container container_1439803298368_0005_01_000003 for
> log-aggregation
> 2015-08-18 01:50:35,089 INFO
> org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices: Got
> event CONTAINER_STOP for appId application_1439803298368_0005
> 2015-08-18 01:50:35,099 INFO
> org.apache.spark.network.yarn.YarnShuffleService: Stopping container
> container_1439803298368_0005_01_000003
> 2015-08-18 01:50:35,105 INFO
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor:
> Deleting absolute path :
> /export/servers/hadoop2.6.0/tmp/nm-local-dir/usercache/root/appcache/application_1439803298368_0005/container_1439803298368_0005_01_000003
> 2015-08-18 01:50:47,601 WARN
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor:* Exit
> code from container container_1439803298368_0005_01_000001 is : 15*
> *2015-08-18 01:50:48,401 WARN
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor:
> Exception from container-launch with container ID:
> container_1439803298368_0005_01_000001 and exit code: 15*
> *ExitCodeException exitCode=15: *
> at org.apache.hadoop.util.Shell.runCommand(Shell.java:538)
> at org.apache.hadoop.util.Shell.run(Shell.java:455)
> at
> org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:715)
> at
> org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:211)
> at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)
> at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:745)
>
>     container_1439803298368_0005_01_000003 was started at 2015-08-17
> 17:26:20. It ran normally. But it transitioned  to succeed at  *2015-08-18
> 01:50:30* . And it transitioned to CONTAINER_STOP in the end.
> container_1439803298368_0005_01_000001 was started at 2015-08-17
> 17:25:42. At *2015-08-18 01:50:48 *it exited suddenly.
>     *At 2015-08-18 14:40:50*, I check the system memory information. It
> shows as follows.
> [root@A01-R08-3-I160-102 logs]# free -m
>              total       used       free     shared    buffers     cached
> Mem:     31983     28927       3055          0        123        103
> -/+ buffers/cache:  28700       3282
> Swap:    16383      15213       1170
>
>     I wonderd that there was too little memory left in the OS a*t
> 2015-08-18 14:40:50.* Is it possilbe  that caused the
> container_1439803298368_0005_01_000001 to be exit?
>     Each spark worker node has 32G memory. The yarn config in
> yarn-site.xml is as follows:
> <property>
>    <name>yarn.nodemanager.resource.memory-mb</name>
>    <value>30720</value>
>    <description>每个节点可用内存,单位MB,默认是8G,spark需要大量内存,这里调整为30g</description>
>   </property>
>   <property>
>    <name>yarn.scheduler.minimum-allocation-mb</name>
>    <value>128</value>
>    <description>单个任务可申请最少内存,默认1024MB,稍微大一点,避免小的计算浪费资源</description>
>   </property>
>   <property>
>    <name>yarn.scheduler.maximum-allocation-mb</name>
>    <value>16384</value>
>    <description>单个任务可申请最大内存,默认8192MB,改为16G,大任务充分利用资源</description>
>   </property>
>
>  30g memory is set for each yarn node. When is the 30g memeory is
> allocated?Is it possilbe that yarn allocates more and more memory as the
> executor runs?
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "Saisai Shao";<sa...@gmail.com>;
> *发送时间:* 2015年8月18日(星期二) 中午11:13
> *收件人:* "刚"<94...@qq.com>;
> *抄送:* "user"<us...@spark.apache.org>;
> *主题:* Re: How should I do to solve this problem that the
> executorsofmysparkapplication always is blocked after an executor is lost?
>
> I think this is actually the log of executor (/export/servers/hadoop2.6.0/
> yarn_logs/application_1439803298368_0003/container_
> 1439803298368_0003_01_000002/stderr). Seems you have a different
> configuration, so the path-to-log might be different.
>
> Another thing you should find out if there's any ERROR or exception in the
> log of node manager. Sometimes it is due to memory issue, the executor is
> killed by node manager, probably you should also check the node manager log.
>
> Thanks
> Saisai
>
>
> On Tue, Aug 18, 2015 at 10:54 AM, 刚 <94...@qq.com> wrote:
>
>> Hi Saisai:
>>     My application has runned for 17 hours. It has not been blocked. But
>> 2 executors have lost. I can not find the stack that cause the executor to
>> exit from the executor log. The log configuration in yarn-site.xml is as
>> follows.
>>  <property>
>>    <name>yarn.nodemanager.log.retain-seconds</name>
>>    <value>36000</value>
>>    <description>Default time (in seconds) to retain log files on the
>> NodeManager Only applicable if log-aggregation is disabled.</description>
>>   </property>
>>
>>   <property>
>>    <name>yarn.log-aggregation.retain-seconds</name>
>>    <value>864000</value>
>>    <description>在HDFS上聚集日志保存的时间,10day=864000 seconds</description>
>>   </property>
>>
>>   <property>
>>    <name>yarn.log-aggregation.retain-check-interval-seconds</name>
>>    <value>-1</value>
>>   </property>
>>
>>   <property>
>>    <name>yarn.nodemanager.remote-app-log-dir</name>
>>    <value>/tmp/logs</value>
>>   </property>
>>
>>   <property>
>>    <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
>>    <value>logs</value>
>>   </property>
>>
>>   <property>
>>      <name>yarn.log-aggregation-enable</name>
>>      <value>true</value>
>>      <description>是否启用日志聚集功能,设置成true</description>
>>   </property>
>>
>>   <property>
>>      <name>*yarn.nodemanager.log-dirs*</name>
>>      <value>*/export/servers/hadoop2.6.0/yarn_logs*</value>
>>      <description>yarn node 日志存放地址</description>
>>   </property>
>>
>>    My hadoop home directory is /export/servers/hadoop2.6.0
>>    The node that executor has lost is 172.19.160.116. So I got the logs
>> from this directory:
>>
>> /export/servers/hadoop2.6.0/yarn_logs/application_1439803298368_0003/container_1439803298368_0003_01_000002/stderr
>> There in no stack in this file.
>> From the yarn-site.xml configuration, I think this file should be  the
>> yarn nodemanager log.
>>
>> And I can not find "userlogs" directory in
>> "/export/servers/hadoop2.6.0/logs" directory.
>> How can I find the logs of executor that has lost? What is the difference
>> between yarn node manager logs and executor logs? Maybe, I think I could
>> find the stack in executor logs.
>>
>>
>>
>>
>>
>>
>>
>> ------------------ 原始邮件 ------------------
>> *发件人:* "Saisai Shao";<sa...@gmail.com>;
>> *发送时间:* 2015年8月17日(星期一) 下午4:51
>> *收件人:* "刚"<94...@qq.com>;
>> *抄送:* "user"<us...@spark.apache.org>;
>> *主题:* Re: How should I do to solve this problem that the
>> executorsofmyspark application always is blocked after an executor is lost?
>>
>> I think this stack is the result, not the reason, this exception is
>> caused by executor lost when trying to connect to executor which has
>> already failed. Would you please dig out to see if there's any other
>> exceptions?
>>
>> Also I'm conservative to the solution of changing to nio, I don't think
>> it is the Netty block transfer service which lead to such problem.
>>
>> Thanks
>> Saisai
>>
>> On Mon, Aug 17, 2015 at 4:34 PM, 刚 <94...@qq.com> wrote:
>>
>>> Hi Saisai:
>>>     Thank you for your reply.
>>>     I find the log of the lost executor, and find the stack as follows:
>>>     15/08/16 13:47:19 ERROR shuffle.OneForOneBlockFetcher: Failed while
>>> starting block fetches
>>> java.io.IOException: Connection reset by peer
>>>         at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>>>         at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>>>         at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>>>         at sun.nio.ch.IOUtil.read(IOUtil.java:192)
>>>         at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:379)
>>>         at
>>> io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:311)
>>>         at
>>> io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881)
>>>         at
>>> io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:225)
>>>         at
>>> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
>>>         at
>>> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>>>         at
>>> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>>>         at
>>> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>>>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>>>         at
>>> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116)
>>>         at java.lang.Thread.run(Thread.java:745)
>>> 15/08/16 13:47:19 INFO shuffle.RetryingBlockFetche
>>>
>>>     And I find how to solve this problem in this link :
>>> http://search-hadoop.com/m/q3RTtss52fWIdgL1
>>>     I have lauched my application by setting spark.shuffle.blockTransferService=nio
>>> to see whether I have solved it.
>>>
>>>
>>>     I have a question that the stack was printed at 15/08/16 13:47:19,
>>> but the exector is lost at 15/08/17 00:32:37. I am not certain whether
>>> there is any relationship between the stack and the blocking of the
>>> application.
>>>
>>>
>>>
>>> ------------------ 原始邮件 ------------------
>>> *发件人:* "Saisai Shao";<sa...@gmail.com>;
>>> *发送时间:* 2015年8月17日(星期一) 中午1:47
>>> *收件人:* "刚"<94...@qq.com>;
>>> *抄送:* "user"<us...@spark.apache.org>;
>>> *主题:* Re: How should I do to solve this problem that the executors
>>> ofmyspark application always is blocked after an executor is lost?
>>>
>>> Here is a link in executor page (stderr), you could click and see the
>>> logs:
>>>
>>>
>>> Also for the failed executors, as far as I know you could log in to the
>>> node where the executor runs and go to the log dir of Yarn, there's a
>>> "userlogs" folder, you could find the log of executors you want.
>>>
>>> For the Yarn node manager log, you could also find out in Yarn log dir.
>>> For HDP it would be /var/log/hadoop-yarn/yarn by default.
>>>
>>> Thanks
>>> Saisai
>>> ​
>>>
>>> On Mon, Aug 17, 2015 at 1:27 PM, 刚 <94...@qq.com> wrote:
>>>
>>>>
>>>>  Hi Saisai:
>>>>     Thank you for your reply. How can I get the log of executors and
>>>> node manager except the yarn log that I have got by using the command "yarn
>>>> logs --applicationId"?
>>>>
>>>> ------------------ 原始邮件 ------------------
>>>> *发件人:* "Saisai Shao";<sa...@gmail.com>;
>>>> *发送时间:* 2015年8月17日(星期一) 中午12:47
>>>> *收件人:* "刚"<94...@qq.com>;
>>>> *抄送:* "user"<us...@spark.apache.org>;
>>>> *主题:* Re: How should I do to solve this problem that the executors of
>>>> myspark application always is blocked after an executor is lost?
>>>>
>>>> Hi,
>>>>
>>>> I think you need to find some clues in the log of executors, as well as
>>>> node manager to dig out more details and exception stacks, so that we could
>>>> get a clear picture of what caused such problems.
>>>>
>>>> Thanks
>>>> Saisai
>>>>
>>>> On Mon, Aug 17, 2015 at 11:35 AM, 刚 <94...@qq.com> wrote:
>>>>
>>>>> Hi guys:
>>>>>     I run 9 applications in my spark-cluster at the same time. They
>>>>> all run well in the beginning. But after several hours, some applications
>>>>> lost one executor, and other executors are blocked. By the way, I am using
>>>>> spark-streaming to analysis real-time messages. The screenshots are as
>>>>> follows.
>>>>>                                                          Figure1: The
>>>>> stage has lasted for long time after one executor is lost
>>>>>
>>>>>
>>>>>                                                   Figure2:The task
>>>>> info of the stage that has last for long time after one executor is lost
>>>>>
>>>>> The command that I submit an application is as follows:
>>>>> spark-submit --class spark_security.login_users.Sockpuppet
>>>>>  --driver-memory 3g --executor-memory 3g --num-executors 3 --executor-cores
>>>>> 4  --name pcLoginSparkDealerUser --master yarn  --deploy-mode cluster
>>>>>  spark_Security-1.0-SNAPSHOT.jar
>>>>> hdfs://A01-R08-3-I160-102.JD.LOCAL:9000/spark_properties/logindelaer.properties
>>>>>
>>>>> Another 8 applications are submitted to use the same driver-memory,
>>>>> executor-memory, num-executors, executor-cores. And they are all run in
>>>>> cluster mode.
>>>>>
>>>>> When the problem happens, I got the yarn logs use the  command as
>>>>> follows:
>>>>>
>>>>> yarn logs -application application_1439457182724_0026
>>>>>
>>>>> I can not find any stack of exception. But I find the information as
>>>>> follows:
>>>>> 15/08/17 00:32:53 INFO streaming.CheckpointWriter: Saving checkpoint
>>>>> for time 1439472653000 ms to file 'hdfs://A01-R08-3-I160-102.JD.LOCAL:9000
>>>>> /regeditCountSparkDealerUser/checkpoint/checkpoint-1439472653000'
>>>>> 15/08/17  00:32:53 INFO streaming.CheckpointWriter: Deleting
>>>>> hdfs://A01-R08-3-I160-102.JD.LOCAL:9000/safemodpasswd/checkpoint/checkpoint-1439472643000
>>>>> 15/08/17  00:32:53 INFO streaming.CheckpointWriter: Checkpoint for
>>>>> time 1439472653000 ms saved to file 'hdfs://A01-R08-3-I160-102.JD.LOCAL:9000
>>>>> /regeditCountSparkDealerUser/checkpoint/checkpoint-1439472653000',
>>>>> took 473939 bytes and 65 ms
>>>>> 15/08/17  00:32:53 INFO transport.ProtocolStateActor: No response
>>>>> from remote. Handshake timed out or transport failure detector triggered.
>>>>> 15/08/17  00:32:53 ERROR cluster.YarnClusterScheduler: Lost executor
>>>>> 5 on A01-R08-2-I160-115.JD.LOCAL: remote Akka client disassociated
>>>>> 15/08/17  00:32:53 WARN remote.ReliableDeliverySupervisor: Association
>>>>> with remote system [akka.tcp://sparkExecutor@A01-R08-2-I160-115.JD.LOCAL:48922]
>>>>> has failed, address is now gated for [5000] ms. Reason is: [Disassociated].
>>>>> 15/08/17 00:32:54 INFO scheduler.TaskSetManager: Re-queueing tasks for
>>>>> 3 from TaskSet 3719.0
>>>>> 15/08/17 00:32:54 INFO dstream.FilteredDStream: Time 1439472654000 ms
>>>>> is invalid as zeroTime is 1439457657000 ms and slideDuration is 15000 ms
>>>>> and difference is 14997000 ms
>>>>> 15/08/17 00:32:54 INFO dstream.FilteredDStream: Time 1439472654000 ms
>>>>> is invalid as zeroTime is 1439457657000 ms and slideDuration is 45000 ms
>>>>> and difference is 14997000 ms
>>>>> 15/08/17 00:32:54 INFO dstream.FilteredDStream: Time 1439472654000 ms
>>>>> is invalid as zeroTime is 1439457657000 ms and slideDuration is 60000 ms
>>>>> and difference is 14997000 ms
>>>>> 15/08/17 00:32:54 INFO dstream.FilteredDStream: Time 1439472654000 ms
>>>>> is invalid as zeroTime is 1439457657000 ms and slideDuration is 120000 ms
>>>>> and difference is 14997000 ms
>>>>> 15/08/17 00:32:54 INFO scheduler.JobScheduler: Added jobs for time
>>>>> 1439472654000 ms
>>>>> 15/08/17 00:32:54 INFO scheduler.JobGenerator: Checkpointing graph for
>>>>> time 1439472654000 ms
>>>>> 15/08/17 00:32:54 INFO streaming.DStreamGraph: Updating checkpoint
>>>>> data for time 1439472654000 ms
>>>>> 15/08/17 00:32:54 WARN scheduler.TaskSetManager: Lost task 11.0 in
>>>>> stage 3719.0 (TID 707634, A01-R08-2-I160-115.JD.LOCAL): ExecutorLostFailure
>>>>> (executor 5 lost)
>>>>> 15/08/17 00:32:54 INFO streaming.DStreamGraph: Updated checkpoint data
>>>>> for time 1439472654000 ms
>>>>> 15/08/17 00:32:54 WARN scheduler.TaskSetManager: Lost task 2.0 in
>>>>> stage 3719.0 (TID 707625, A01-R08-2-I160-115.JD.LOCAL): ExecutorLostFailure
>>>>> (executor 5 lost)
>>>>> 15/08/17 00:32:54 WARN scheduler.TaskSetManager: Lost task 5.0 in
>>>>> stage 3719.0 (TID 707628, A01-R08-2-I160-115.JD.LOCAL): ExecutorLostFailure
>>>>> (executor 5 lost)
>>>>> 15/08/17 00:32:54 WARN scheduler.TaskSetManager: Lost task 8.0 in
>>>>> stage 3719.0 (TID 707631, A01-R08-2-I160-115.JD.LOCAL): ExecutorLostFailure
>>>>> (executor 5 lost)
>>>>> 15/08/17  00:32:54 INFO scheduler.DAGScheduler: Executor lost: 3
>>>>> (epoch 930)
>>>>> 15/08/17  00:32:54 INFO storage.BlockManagerMasterActor: Trying to
>>>>> remove executor 3 from BlockManagerMaster.
>>>>> 15/08/17  00:32:54 INFO storage.BlockManagerMaster: Removed 3
>>>>> successfully in removeExecutor
>>>>> 15/08/17  00:32:54 INFO scheduler.Stage: Stage 3718 is now unavailable
>>>>> on executor 3 (111/180, false)
>>>>> 15/08/17  00:32:54 INFO streaming.CheckpointWriter: Saving checkpoint
>>>>> for time 1439472654000 ms to file 'hdfs://A01-R08-3-I160-102.JD.LOCAL:90
>>>>> 00/regeditCountSparkDealerUser/checkpoint/checkpoint-1439472654000'
>>>>>
>>>>> Some one says that it is caused by OOM, but I can not find any stack
>>>>> of OOM.
>>>>>
>>>>> I set the spark-defaults.con as follows:
>>>>> spark.core.connection.ack.wait.timeout  3600
>>>>> spark.core.connection.auth.wait.timeout 3600
>>>>> spark.akka.frameSize                    1024
>>>>> spark.driver.extraJavaOptions           -Dhdp.version=2.2.0.0–2041
>>>>> spark.yarn.am.extraJavaOptions          -Dhdp.version=2.2.0.0–2041
>>>>> spark.akka.timeout                      900
>>>>> spark.storage.memoryFraction            0.4
>>>>> spark.rdd.compress
>>>>>
>>>>> It is very appreciated that anyone can tell me how to solve this
>>>>> problem. It has botherd me for a long time.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>
>>>
>>
>