Re: Reset of transient variables in state to default values.

2021-10-20 Thread Yun Tang
From: Arvid Heise Sent: Monday, October 18, 2021 20:30 To: Alex Drobinsky Cc: JING ZHANG ; Yun Tang ; User-Flink Subject: Re: Reset of transient variables in state to default values. That's what I would try out, but I'm not sure if the statebackend would pick that up.

Re: Reset of transient variables in state to default values.

2021-10-18 Thread Arvid Heise
;>>> accessing RocksDB state or checkpoint via FileSystemStateBackend, and I >>>>> don't know whether Kryo would serialize your transient field. >>>>> If you're not familiar with Flink's serialization stack, I think you >>>>> coul

Re: Reset of transient variables in state to default values.

2021-10-18 Thread Alex Drobinsky
, I think you >>>> could check behaviors below: >>>> >>>>1. Without any checkpoint restore, use FileSystemStateBackend to >>>>see whether the transient field could be read as expected, the answer >>>>should be yes. >>>>

Re: Reset of transient variables in state to default values.

2021-10-18 Thread Arvid Heise
eckpoint, check whether could read the >>>transient field back if using FileSystemStateBackend. >>> >>> >>> >>> [1] >>> https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/dev/serialization/types_serialization/#flinks-type

Re: Reset of transient variables in state to default values.

2021-10-12 Thread Alex Drobinsky
> >> [1] >> https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/dev/serialization/types_serialization/#flinks-typeinformation-class >> >> Best >> Yun Tang >> >> >> -- >> *From:* Alex Drobinsky >&

Re: Reset of transient variables in state to default values.

2021-10-12 Thread JING ZHANG
1.13/docs/dev/serialization/types_serialization/#flinks-typeinformation-class > > Best > Yun Tang > > > -- > *From:* Alex Drobinsky > *Sent:* Monday, October 11, 2021 22:37 > *To:* JING ZHANG > *Cc:* User-Flink > *Subject:* Re: Reset o

Re: Reset of transient variables in state to default values.

2021-10-12 Thread Yun Tang
Monday, October 11, 2021 22:37 To: JING ZHANG Cc: User-Flink Subject: Re: Reset of transient variables in state to default values. It would be difficult to provide even a semblance of the complete product , however I could try to provide enough details to reproduce the problem. Standard

Re: Reset of transient variables in state to default values.

2021-10-11 Thread Alex Drobinsky
It would be difficult to provide even a semblance of the complete product , however I could try to provide enough details to reproduce the problem. Standard source would do: DataStream stream = env.addSource( new FlinkKafkaConsumer<>(topic, new AbstractDeserializationSchema() {

Re: Reset of transient variables in state to default values.

2021-10-11 Thread JING ZHANG
Hi Alex, It is a little weird. Would you please provide the program which could reproduce the problem, including DataStream job code and related classes code. I need some debug to find out the reason. Best, JING ZHANG Alex Drobinsky 于2021年10月11日周一 下午5:50写道: > Hi Jing Zhang, > > I'm using the F

Re: Reset of transient variables in state to default values.

2021-10-11 Thread Alex Drobinsky
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 MultiStorePac

Re: Reset of transient variables in state to default values.

2021-10-11 Thread JING ZHANG
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 though

Reset of transient variables in state to default values.

2021-10-11 Thread Alex Drobinsky
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; p