flink-user-zh mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Xintong Song <tonysong...@gmail.com>
Subject Re: exception:beyond the 'PHYSICAL' memory limit
Date Fri, 08 May 2020 08:10:00 GMT
也可以尝试按照下面的方法进一步调大 native 内存。
1. taskmanager.memory.task.heap.size 进一步调小,比如 1.5g 或 1g
2. taskmanager.memory.jvm-overhead.max 调大以避免冲突,建议先调成 4g

另外你说这个问题是作业运行一段时间后发生的,具体是多长时间发生的呢?也可以考虑先把
-ytm 调得很大,观察一下 container
的内存用量是如何增长的。如果持续增长不收敛,那有可能是存在内存泄漏的问题。

Thank you~

Xintong Song



On Fri, May 8, 2020 at 3:56 PM Xintong Song <tonysong820@gmail.com> wrote:

> 这个报错是 Flink 使用的内存超出了申请的 container 大小,被 yarn 杀掉了。因为
jvm 没有报 oom,所以应该只能是
> native 内存超用。
>
> 看了一下你的配置,按照这个配置 jvm overhead 留出了 800MB+,再加上按你说的
streaming 作业没有用到 state,那
> 512 MB 的 manage memory 也空出来了,加在一起 native 内存有 1.3GB+ 应该是够用的。从
dump
> 信息来看配置是正常生效了,也没有其他子进程,按理说不应该超用才对。
>
> 建议还是再确认下是否有用到了 rocksdb,以及是否有用到其他使用
native 内存的 libaray。
>
> Thank you~
>
> Xintong Song
>
>
>
> On Fri, May 8, 2020 at 1:16 PM tiantingting5435@163.com <
> tiantingting5435@163.com> wrote:
>
>> 最近用flink1.10写一个流任务,大致的逻辑,是将数据清洗以后写入hbase,中间不需要保存任何状态
>> 这里是启动脚本:
>> export HADOOP_CONF_DIR=/etc/hadoop/conf
>> export HADOOP_CLASSPATH=/opt/cloudera/parcels/CDH/jars/*
>>
>> /opt/flink/flink-1.10.0/bin/flink run -p 1 -m yarn-cluster -yqu
>> root.crawler \
>>   -ynm testRealTime22 \
>>   -yD taskmanager.debug.memory.log=true \
>>   -yD taskmanager.debug.memory.log-interval=10000 \
>>   -yD yarn.nodemanager.vmem-check-enabled=false \
>>   -yD taskmanager.memory.task.heap.size=2049m \
>>   -yD taskmanager.memory.managed.size=512m \
>>   -ys 1 -yjm 2048m -ytm 4096m -d \
>>   -c com.fengjr.risk.BorrowerRelatedRiskApp
>> /home/dev/test/real_time/risk-principal-borrower-related-1.0-beta.jar
>> 1000000
>>
>> 但是,程序总是在运行的一段时间后被yarn kill,异常如下:
>> Closing TaskExecutor connection
>> container_e16_1571280155773_0449_01_000002 because: [2020-05-08
>> 12:23:57.963]Container
>> [pid=40959,containerID=container_e16_1571280155773_0449_01_000002] is
>> running 102236160B beyond the 'PHYSICAL' memory limit. Current usage: 4.1
>> GB of 4 GB physical memory used; 7.6 GB of 8.4 GB virtual memory used.
>> Killing container.
>> Dump of the process-tree for container_e16_1571280155773_0449_01_000002 :
>> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS)
>> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
>> |- 40959 40956 40959 40959 (bash) 0 1 11964416 351 /bin/bash -c
>> /usr/java/jdk1.8.0_201-amd64/bin/java
>> -Xmx2282749952
>> -Xms2282749952
>> -XX:MaxDirectMemorySize=462422021
>> -XX:MaxMetaspaceSize=100663296
>> -Dfile.encoding=UTF-8
>> -Dlog.file=/yarn/container-logs/application_1571280155773_0449/container_e16_1571280155773_0449_01_000002/taskmanager.log
>> -Dlog4j.configuration=file:./log4j.properties
>> org.apache.flink.yarn.YarnTaskExecutorRunner -D
>> taskmanager.memory.framework.off-heap.size=134217728b -D
>> taskmanager.memory.network.max=328204293b -D
>> taskmanager.memory.network.min=328204293b -D
>> taskmanager.memory.framework.heap.size=134217728b -D
>> taskmanager.memory.managed.size=536870912b -D taskmanager.cpu.cores=1.0 -D
>> taskmanager.memory.task.heap.size=2148532224b -D
>> taskmanager.memory.task.off-heap.size=0b --configDir .
>> -Djobmanager.rpc.address='fjr-yz-0-135' -Dweb.port='0'
>> -Dweb.tmpdir='/tmp/flink-web-b7d8e065-7809-4918-baa1-6be5ce4f8a03'
>> -Djobmanager.rpc.port='27863' -Drest.address='fjr-yz-0-135'
>> -Dsecurity.kerberos.login.keytab='/yarn/nm/usercache/hbase/appcache/application_1571280155773_0449/container_e16_1571280155773_0449_01_000001/krb5.keytab'
>> 1>
>> /yarn/container-logs/application_1571280155773_0449/container_e16_1571280155773_0449_01_000002/taskmanager.out
>> 2>
>> /yarn/container-logs/application_1571280155773_0449/container_e16_1571280155773_0449_01_000002/taskmanager.err
>>
>> |- 326 40959 40959 40959 (java) 27897 4892 8097333248 1073185
>> /usr/java/jdk1.8.0_201-amd64/bin/java -Xmx2282749952 -Xms2282749952
>> -XX:MaxDirectMemorySize=462422021 -XX:MaxMetaspaceSize=100663296
>> -Dfile.encoding=UTF-8
>> -Dlog.file=/yarn/container-logs/application_1571280155773_0449/container_e16_1571280155773_0449_01_000002/taskmanager.log
>> -Dlog4j.configuration=file:./log4j.properties
>> org.apache.flink.yarn.YarnTaskExecutorRunner -D
>> taskmanager.memory.framework.off-heap.size=134217728b -D
>> taskmanager.memory.network.max=328204293b -D
>> taskmanager.memory.network.min=328204293b -D
>> taskmanager.memory.framework.heap.size=134217728b -D
>> taskmanager.memory.managed.size=536870912b -D taskmanager.cpu.cores=1.0 -D
>> taskmanager.memory.task.heap.size=2148532224b -D
>> taskmanager.memory.task.off-heap.size=0b --configDir .
>> -Djobmanager.rpc.address=fjr-yz-0-135 -Dweb.port=0
>> -Dweb.tmpdir=/tmp/flink-web-b7d8e065-7809-4918-baa1-6be5ce4f8a03
>> -Djobmanager.rpc.port=27863 -Drest.address=fjr-yz-0-135
>> -Dsecurity.kerberos.login.keytab=/yarn/nm/usercache/hbase/appcache/application_1571280155773_0449/container_e16_1571280155773_0449_01_000001/krb5.keytab
>>
>>
>> [2020-05-08 12:23:58.020]Container killed on request. Exit code is 143
>> [2020-05-08 12:23:58.035]Container exited with a non-zero exit code 143.
>>
>>
>> tiantingting5435@163.com
>>
>
Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message