[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2019-01-28 Thread Dawid Wysakowicz (JIRA)


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

Dawid Wysakowicz commented on FLINK-10928:
--

Hi [~djharper],
I think you managed to find the core issue as discussed here: 
https://lists.apache.org/thread.html/fb2eb17c272a53686c28448c437d46516e01ccc0cf67fd5ca9ee794e@%3Cuser.flink.apache.org%3E,
 right?

Would you be ok with closing this issue?

> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
> Attachments: Screen Shot 2018-11-16 at 15.49.03.png, Screen Shot 
> 2018-11-16 at 15.49.15.png, Screen Shot 2018-12-10 at 14.13.52.png, 
> ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf
>
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) 
> I've attached a timeline of events as a PDF here  
> [^ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf]  but essentially 
> it boils down to
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_0001_01_000716/taskmanager.err
> |- 7738 7725 7725 7725 (java) 6959576 976377 8904458240 1671684 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCL
> ogFileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652
> _0001/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir .
>  
> Container killed on request. Exit code is 143
> Container exited with a non-zero exit code 143
> {code}
> 6. YARN allocates new containers but the job is never able to get back into a 
> stable state, with constant restarts until eventually the job is cancelled 
> We've seen something similar to FLINK-10848 happening to with some task 
> managers allocated but sitting 'idle' state. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2018-12-14 Thread Daniel Harper (JIRA)


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

Daniel Harper commented on FLINK-10928:
---

Hi [~dawidwys] 

Going to be tricky to provide a heap dump due to sensitive data unfortunately.

We've resolved the connection timeouts issue by increasing the connection pool 
size from 15 to 30, which after running this for 7 days or so has resulted in 0 
'timeout waiting for connection from pool' errors when the job restarts and 
restores from a checkpoint

One of the causes of the job restarting in the first place is FLINK-10844, 
which causes the checkpoint to fail (note this is intermittent we see this once 
or twice a day) which causes the job to restart. 

We are looking at enabling the setting {{failOnCheckpointingErrors}} to false 
to mitigate this in the meantime, although we understand the risk in enabling 
this setting.

This 'death spiral'/instability has happened 3 or so times in the past 6 weeks, 
and we see the job restarting once or twice a day in the times between these 
massive failures. The only thing I can think of is a memory leak building over 
time and eventually triggering YARN to kill the containers.

I did a heap dump on one of the taskmanagers this morning and it looks to me 
like there are multiple copies of 'user' classes i.e. BEAM code and our code, 
most of which have 0 instances, which looks like a classloader leak to me? This 
snapshot was taken after the job had restarted about 10 times. 

 !Screen Shot 2018-12-10 at 14.13.52.png! 


> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
> Attachments: Screen Shot 2018-11-16 at 15.49.03.png, Screen Shot 
> 2018-11-16 at 15.49.15.png, 
> ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf
>
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) 
> I've attached a timeline of events as a PDF here  
> [^ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf]  but essentially 
> it boils down to
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_0001_01_000716/taskmanager.err
> |- 7738 7725 7725 7725 (java) 6959576 976377 8904458240 1671684 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCL
> ogFileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652
> _0001/container_1541433014652_0001_01_000716/taskmanager.lo

[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2018-12-03 Thread Dawid Wysakowicz (JIRA)


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

Dawid Wysakowicz commented on FLINK-10928:
--

Hi [~djharper]
Where you able to figure out the issue? I would say there are to separate 
problems:
1. Ever growing metaspace size resulting in yarn containers being killed
- could you provide us with a heap dump of your job, so that we could try to 
analyze why the classes are not being GCed?
2. Connection problem that results in job restarts
- I would tackle this problem after resolving the first one.

> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
> Attachments: Screen Shot 2018-11-16 at 15.49.03.png, Screen Shot 
> 2018-11-16 at 15.49.15.png, 
> ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf
>
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) 
> I've attached a timeline of events as a PDF here  
> [^ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf]  but essentially 
> it boils down to
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_0001_01_000716/taskmanager.err
> |- 7738 7725 7725 7725 (java) 6959576 976377 8904458240 1671684 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCL
> ogFileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652
> _0001/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir .
>  
> Container killed on request. Exit code is 143
> Container exited with a non-zero exit code 143
> {code}
> 6. YARN allocates new containers but the job is never able to get back into a 
> stable state, with constant restarts until eventually the job is cancelled 
> We've seen something similar to FLINK-10848 happening to with some task 
> managers allocated but sitting 'idle' state. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2018-11-19 Thread Biao Liu (JIRA)


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

Biao Liu commented on FLINK-10928:
--

Hi [~djharper]

1. "Why does YARN kill the containers with out of memory?"

The reason is described clearly in exception. 
Container [pid=7725,containerID=container_1541433014652_0001_01_
000716] is running beyond physical memory limits. Current usage: 6.4 GB of 6.4 
GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
container.
Your container is beyond physical memory limits. Not because OOM, OOM may cause 
failure, but not being killed.

 

2. "Is it possible for the task manager to allocate memory outside of the 'off 
heap' allocation, which would cause YARN to kill the container?"

Yes, it is possible. JVM, state backend, Netty, all these components may 
allocate off heap memory or native memory.

 

3. "Why do we get timeout waiting for connection from pool from the AWS SDK?"

I'm not sure because I can't see the whole picture of your job. However there 
is a "FileNotFoundException" which is thrown by user code. I think that's not 
caused by Flink, right?

 
{code:java}
Caused by: org.apache.beam.sdk.util.UserCodeException: 
java.io.FileNotFoundException: Reopen at position 0 on 
s3a://.../beam/.temp-beam-2018-11-05_15-54-26-0/bc47b14b-1679-45ce-81b7- 
a4d19e036cb5: com.amazonaws.services.s3.model.AmazonS3Exception: The specified 
key does not exist. (Service: Amazon S3; Status Code: 404; Error Code: 
NoSuchKey; Request ID: 0D67ACD1037E5B52; S3 Extended Request ID: 
BVgqzksS75Dv1EkZyUgkVMl8brE1PznBM1RsN9uXp2cnn8Rf+r+b9D09TWZQtpW8aSbQi7R9 RW8=), 
S3 Extended Request ID: 
BVgqzksS75Dv1EkZyUgkVMl8brE1PznBM1RsN9uXp2cnn8Rf+r+b9D09TWZQtpW8aSbQi7R9 RW8=
{code}
 

There are too many problems in your description. Most of them seem to be 
nothing related with Flink framework. 

Could you fix the memory and the FileNotFoundException first? 

 

> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
> Attachments: Screen Shot 2018-11-16 at 15.49.03.png, Screen Shot 
> 2018-11-16 at 15.49.15.png, 
> ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf
>
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) 
> I've attached a timeline of events as a PDF here  
> [^ants-CopyofThe'death'spiralincident-191118-1231-1332.pdf]  but essentially 
> it boils down to
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_0001_01_000716/taskmanager.err
> |- 7738 7725 7725 7725 (java) 6959576 976377 8904458240 1671684 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:

[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2018-11-19 Thread Daniel Harper (JIRA)


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

Daniel Harper commented on FLINK-10928:
---

h2. Why do we get timeout waiting for connection from pool from the AWS SDK? 

We're still investigating this area.

It sounds like either there's connection leakage when a job restarts and 
doesn't clean up properly, or we have sized our connection pool too low.

> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
> Attachments: Screen Shot 2018-11-16 at 15.49.03.png, Screen Shot 
> 2018-11-16 at 15.49.15.png
>
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) but essentially we've seen the following sequence of 
> events: 
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_0001_01_000716/taskmanager.err
> |- 7738 7725 7725 7725 (java) 6959576 976377 8904458240 1671684 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCL
> ogFileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652
> _0001/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir .
>  
> Container killed on request. Exit code is 143
> Container exited with a non-zero exit code 143
> {code}
> 6. YARN allocates new containers but the job is never able to get back into a 
> stable state, with constant restarts until eventually the job is cancelled 
> We've seen something similar to FLINK-10848 happening to with some task 
> managers allocated but sitting 'idle' state. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2018-11-19 Thread Daniel Harper (JIRA)


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

Daniel Harper commented on FLINK-10928:
---

h1. Why does YARN kill the containers with out of memory

We run FLINK on EMR with the following memory settings:

{code}
--taskManagerMemory 6500 
--jobManagerMemory 6272 
--detached 
-Dcontainerized.heap-cutoff-ratio=0.15 
-Dclassloader.resolve-order=parent-first 
-Dparallelism.default=32 
-Dstate.backend=filesystem 
-Dyarn.maximum-failed-containers=-1 
-Djobmanager.web.checkpoints.history=1000 
"-Dakka.ask.timeout=60 s" 
"-Denv.java.opts=-Xloggc:/var/log/hadoop-yarn/flink_gc_$(basename  | 
egrep -o 'container_[0-9]+_[0-9]+')_%p.log -XX:GCLogFileSize=200M 
-XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps 
-XX:+PrintTenuringDistribution -XX:+PrintGCCause -XX:+PrintGCDateStamps 
-XX:+UseG1GC" 
-Dstate.backend.fs.checkpointdir=s3a://...
-Dstate.checkpoints.dir=s3a://...
-Dstate.savepoints.dir=s3a://...
{code}

Through YARN we can see that each container gets allocated with 6528mb (heap 
4995mb, off heap 1533mb)

The question is, why does the YARN container get killed after a few restarts? 

One avenue I investigated was restricting the s3 connection pool size for 
hadoop to force it to restart

h2. Simulating restarts on TEST

After deploying this to TEST we observed the following on one of the task 
managers by connecting via JMX

* Upon each restart the metaspace size + number of classes loaded increased 
* Prior to YARN killing the container, the job was restarting roughly every 30 
seconds which seemed to accelerate the metaspace size used

Screenshots from JVISUALVM shown below

heap
!Screen Shot 2018-11-16 at 15.49.15.png! 

metaspace
!Screen Shot 2018-11-16 at 15.49.03.png!  

h2. Is this a problem?

This is what we are not sure about. 

Is it possible for the task manager to allocate memory outside of the 'off 
heap' allocation, which would cause YARN to kill the container?

The metaspace size is currently unbounded so I am making the assumption this is 
the cause, but I'm happy to be corrected otherwise.

I noticed there was a ticket FLINK-10317 related to setting an upper bound to 
the metaspace size but it looks like there's some concern about what to set 
this to. 

> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
> Attachments: Screen Shot 2018-11-16 at 15.49.03.png, Screen Shot 
> 2018-11-16 at 15.49.15.png
>
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) but essentially we've seen the following sequence of 
> events: 
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_00

[jira] [Commented] (FLINK-10928) Job unable to stabilise after restart

2018-11-19 Thread Daniel Harper (JIRA)


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

Daniel Harper commented on FLINK-10928:
---

I recognise this is quite vague, we've been trying to reproduce this on our 
TEST environment.

The two areas we've tried to focus on are

1. Why does YARN kill the containers with out of memory?
2. Why do we get timeout waiting for connection from pool from the AWS SDK? 


> Job unable to stabilise after restart 
> --
>
> Key: FLINK-10928
> URL: https://issues.apache.org/jira/browse/FLINK-10928
> Project: Flink
>  Issue Type: Bug
> Environment: AWS EMR 5.17.0
> FLINK 1.5.2
> BEAM 2.7.0
>Reporter: Daniel Harper
>Priority: Major
>
> We've seen a few instances of this occurring in production now (it's 
> difficult to reproduce) but essentially we've seen the following sequence of 
> events: 
> 1. Job restarts due to exception
> 2. Job restores from a checkpoint but we get the exception
> {code}
> Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: 
> Timeout waiting for connection from pool
> {code}
> 3. Job restarts
> 4. Job restores from a checkpoint but we get the same exception
>  repeat a few times within 2-3 minutes
> 5. YARN kills containers with out of memory
> {code}
> 2018-11-14 00:16:04,430 INFO  org.apache.flink.yarn.YarnResourceManager   
>   - Closing TaskExecutor connection 
> container_1541433014652_0001_01_000716 because: Container 
> [pid=7725,containerID=container_1541433014652_0001_01_
> 000716] is running beyond physical memory limits. Current usage: 6.4 GB of 
> 6.4 GB physical memory used; 8.4 GB of 31.9 GB virtual memory used. Killing 
> container.
> Dump of the process-tree for container_1541433014652_0001_01_000716 :
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS) 
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
> |- 7725 7723 7725 7725 (bash) 0 0 115863552 696 /bin/bash -c 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCLogF
> ileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652_00
> 01/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir . 1> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container
> _1541433014652_0001_01_000716/taskmanager.out 2> 
> /var/log/hadoop-yarn/containers/application_1541433014652_0001/container_1541433014652_0001_01_000716/taskmanager.err
> |- 7738 7725 7725 7725 (java) 6959576 976377 8904458240 1671684 
> /usr/lib/jvm/java-openjdk/bin/java -Xms4995m -Xmx4995m 
> -XX:MaxDirectMemorySize=1533m 
> -Xloggc:/var/log/hadoop-yarn/flink_gc_container_1541433014652_0001_%p.log 
> -XX:GCL
> ogFileSize=200M -XX:NumberOfGCLogFiles=10 -XX:+PrintGCDetails 
> -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCCause 
> -XX:+PrintGCDateStamps -XX:+UseG1GC 
> -Dlog.file=/var/log/hadoop-yarn/containers/application_1541433014652
> _0001/container_1541433014652_0001_01_000716/taskmanager.log 
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskExecutorRunner --configDir .
>  
> Container killed on request. Exit code is 143
> Container exited with a non-zero exit code 143
> {code}
> 6. YARN allocates new containers but the job is never able to get back into a 
> stable state, with constant restarts until eventually the job is cancelled 
> We've seen this occurring too FLINK-10848 with some task managers allocated 
> but sitting 'idle' state. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)