Hi Jing Zhang,

I'm using the FileSystem backend. I also implemented ReadObject function to
support proper restart procedure:

private void readObject(ObjectInputStream ois)
        throws ClassNotFoundException, IOException {
    ois.defaultReadObject();
    logger.info("Deserialized MultiStorePacketState: " + this.toString());

    // No need to do anything in case of empty file
    if (fileName.isEmpty()) {
        return;
    }
    currentFile = new RandomAccessFile(fileName,"rw");
    currentFile.seek(fileOffset);
}

However, according to logs this function wasn't called.

Btw, it could be beneficial to add this kind of State object e.g.
FileState which will encapsulate serialization / deserialization for
RandomAccessFile although the concept itself is a bit contradictory to
regular state.

Currently, I implemented and tested a workaround via addition of the
boolean variable isFileOpened, however it's awkward because I need to
check the state of the transient variable every time I use
state.value().

So should it be expected that transient variables in state would be
resetted to default values ?


пн, 11 окт. 2021 г. в 12:33, JING ZHANG <beyond1...@gmail.com>:

> 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
>>
>>

Reply via email to