You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Rinat <r....@cleverdata.ru> on 2017/10/20 15:22:29 UTC

BucketingSink with disabled checkpointing will never clean up it's state

Hi, got one more little question about BucketingSink with disabled checkpointing.
In terms of my current task, I’m looking through sources of BucketingSink and it seem’s that I found an issue for the case, when checkpointing is disabled.

BucketingSink - is a flink rich function, that also implements checkpointing interface, so, it stores all metadata about open writers in it’s state.
On invoke method call, we creates new writers, if it’s necessary to open file with a new path, and on writer creation we adds it to the state meta info. 

Later, when file size exceeds the configured limit or timer tells us, that open writers should be closed, sink closes writers, moves file into pending state, and updates it’s state by adding file name of closed writer
to the collection of pending files.

Later, if checkpointing is enabled, sink will be notified, and it’ll move all pending files from pending to final state and clean up it’s current state.
But, what happens, if checkpointing is disabled ? 

I looked through the code base of BucketingSink, seems to me, the state will be never cleaned up and number of managed files will always grow, when checkpointing is disabled.
I could provide more information if necessary, currently, my suspicions are based only on the code base research

Do anyone uses BucketingSink with disabled checkpointing in prod environment ?

Thx.