Re: Restoring 1.3.1 savepoint in 1.4.0 fails in TimerService

2018-01-12 Thread Tzu-Li (Gordon) Tai
Thanks a lot for looking into this with so much detail, Juho! It was super helpful. Shortly put: This is indeed a bug with Flink. The HeapInternalTimerService should be performing compatibility checks on the restored / provided serializers and reconfigure serializers if possible, instead of jus

Re: Restoring 1.3.1 savepoint in 1.4.0 fails in TimerService

2018-01-12 Thread Juho Autio
Thanks, the window operator is just: .timeWindow(Time.seconds(10)) We haven't changed key types. I tried debugging this issue in IDE and found the root cause to be this: !this.keyDeserializer.equals(keySerializer) -> true => throw new IllegalStateException("Tried to initialize restored TimerS

Re: Restoring 1.3.1 savepoint in 1.4.0 fails in TimerService

2018-01-12 Thread Tzu-Li (Gordon) Tai
Hi Juho, Could your key type have possibly changed / been modified across the upgrade? Also, from the error trace, it seems like the failing restore is of a window operator. What window type are you using? That exception is a result of either mismatching key serializers or namespace serializers (

Restoring 1.3.1 savepoint in 1.4.0 fails in TimerService

2018-01-12 Thread Juho Autio
I'm trying to restore savepoints that were made with Flink 1.3.1 but getting this exception. The few code changes that had to be done to switch to 1.4.0 don't seem to be related to this, and it seems like an internal issue of Flink. Is 1.4.0 supposed to be able to restore a savepoint that was made