Hi, Alex What state backend do you choose? If you choose MemoryStateBackend or FsStateBackend, `transient` keyword may not have effect because MemoryStateBackend does not serialize state for regular read/write accesses but keeps it as objects on the heap. If you choose RocksDBStateBackend, I thought it was expected behavior because RocksDBStateBackend stores all state as byte arrays in embedded RocksDB instances. Therefore, it de/serializes the state of a key for every read/write access. CurrentFile is null because the transient variable would not be serialized by default.
Best, JING ZHANG Alex Drobinsky <alex.drobin...@gmail.com> 于2021年10月11日周一 下午4:33写道: > Dear flink community, > > I have following state class ( irrelevant fields removed ) > public class MultiStorePacketState implements Serializable { > > public transient RandomAccessFile currentFile = null; > public long timerValue; > public String fileName; > public String exportedFileName; > public String sessionKey; > public long fileOffset = 0; > > } > > Once in a while, currentFile became *nullified, *this happens after I > extract state via > > MultiStorePacketState so = state.value(); > > The frequency of this behaviour is similar to checkpoint interval ( > checkpoint interval defined as 5 seconds and first occurence of this problem > is also 5 seconds), otherwise I don't have any clues to a possible > explanation. > > Is it an expected side effect of checkpoint procedure ? > > Best regards, > Alex > >