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

Xintong Song edited comment on FLINK-15900 at 2/4/20 3:46 PM:
--------------------------------------------------------------

I also looked into this issue, and offline discussed with [~jark] and [~yunta].

I could not reproduce this locally either.

This issue does not seems to me that rocksdb acquired memory and failed to 
release at job failover.
* The exception shows that rocksdb failed to reserve memory from memory 
manager, because there's no memory left in the memory manager. See 
{{MemoryManager#getSharedMemoryResourceForManagedMemory}}.
* The code path where the exception is thrown is for rocksdb reserving memory 
from memory manager. For this code path, rocksdb allocates a shared resource 
from memory manager, in a lazy bias. That means:
** Rocksdb will always use previous allocated shared resource as long as it 
exist, and will only try to reserve memory from memory manager if the share 
resource is not allocated.
** Even if rocksdb has failed to release the memory when job failed, when it 
tries to allocate memory after failover, it should get the same previous 
allocated shared resources and should not try to reserve memory from memory 
manager again.
* I looked in to the codes in {{SharedResources}}, where the lazy allocation 
memory sharing logic is implemented, and did not find any problem. The class is 
thread safe. 

Maybe [~sewen] as the original contributor can double check on that.

Another direction of suspicion that I can come up with is that, the managed 
memory size might be configured to 0 in some specific / unstable cases. I do 
not find any evidence for this either. However, if this is indeed the case, it 
also explains the failures in related FLINK-15619 and FLINK-15705.


was (Author: xintongsong):
I also looked into this issue, and offline discussed with [~jark] and [~yunta].

I could reproduce this locally either.

This issue does not seems to me that rocksdb acquired memory and failed to 
release at job failover.
* The exception shows that rocksdb failed to reserve memory from memory 
manager, because there's no memory left in the memory manager. See 
{{MemoryManager#getSharedMemoryResourceForManagedMemory}}.
* The code path where the exception is thrown is for rocksdb reserving memory 
from memory manager. For this code path, rocksdb allocates a shared resource 
from memory manager, in a lazy bias. That means:
** Rocksdb will always use previous allocated shared resource as long as it 
exist, and will only try to reserve memory from memory manager if the share 
resource is not allocated.
** Even if rocksdb has failed to release the memory when job failed, when it 
tries to allocate memory after failover, it should get the same previous 
allocated shared resources and should not try to reserve memory from memory 
manager again.
* I looked in to the codes in {{SharedResources}}, where the lazy allocation 
memory sharing logic is implemented, and did not find any problem. The class is 
thread safe. 

Maybe [~sewen] as the original contributor can double check on that.

Another direction of suspicion that I can come up with is that, the managed 
memory size might be configured to 0 in some specific / unstable cases. I do 
not find any evidence for this either. However, if this is indeed the case, it 
also explains the failures in related FLINK-15619 and FLINK-15705.

> JoinITCase#testRightJoinWithPk failed on Travis
> -----------------------------------------------
>
>                 Key: FLINK-15900
>                 URL: https://issues.apache.org/jira/browse/FLINK-15900
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.10.0
>            Reporter: Gary Yao
>            Priority: Critical
>              Labels: test-stability
>             Fix For: 1.10.0
>
>
> {noformat}
> org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
>       at 
> org.apache.flink.table.planner.runtime.stream.sql.JoinITCase.testRightJoinWithPk(JoinITCase.scala:672)
> Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by 
> FixedDelayRestartBackoffTimeStrategy(maxNumberRestartAttempts=1, 
> backoffTimeMS=0)
> Caused by: java.lang.Exception: Exception while creating 
> StreamOperatorStateContext.
> Caused by: org.apache.flink.util.FlinkException: Could not restore keyed 
> state backend for KeyedProcessOperator_17aecc34cf8aa256be6fe4836cbdf29a_(2/4) 
> from any of the 1 provided restore options.
> Caused by: java.io.IOException: Failed to acquire shared cache resource for 
> RocksDB
> Caused by: org.apache.flink.runtime.memory.MemoryAllocationException: Could 
> not created the shared memory resource of size 20971520. Not enough memory 
> left to reserve from the slot's managed memory.
> Caused by: org.apache.flink.runtime.memory.MemoryReservationException: Could 
> not allocate 20971520 bytes. Only 0 bytes are remaining.
> {noformat}
> https://api.travis-ci.org/v3/job/645466432/log.txt



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to