Restore state class not found exception in 1.8

classic Classic list List threaded Threaded
7 messages Options
Reply | Threaded
Open this post in threaded view
|

Restore state class not found exception in 1.8

Lasse Nedergaard
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more

 
Reply | Threaded
Open this post in threaded view
|

Re: Restore state class not found exception in 1.8

Tzu-Li (Gordon) Tai
Hi Lasse,

Did you move the class to a different namespace / package or changed to be a nested class, across the Flink versions?
That would be the only cause I could reason about at the moment.

If possible, could you also have a very minimal snippet / instructions on how I can maybe reproduce this?
That might give me more insight.

Cheers,
Gordon

On Mon, May 27, 2019 at 7:52 PM Lasse Nedergaard <[hidden email]> wrote:
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more

 
Reply | Threaded
Open this post in threaded view
|

Re: Restore state class not found exception in 1.8

Lasse Nedergaard
Hi Gordon

Thanks for the reply. No we haven’t moved it around namespaces. The only thing we have done is to add a new attribute to the object in another branch of our code and it could be we by mistake has used it but it should still not give a class not found exception. 
We have the save point data in S3 so is there a way to use this save point together with test case so we can debug it locally? Or start Flink mini cluster with this save point?

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 28. maj 2019 kl. 10.06 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

Did you move the class to a different namespace / package or changed to be a nested class, across the Flink versions?
That would be the only cause I could reason about at the moment.

If possible, could you also have a very minimal snippet / instructions on how I can maybe reproduce this?
That might give me more insight.

Cheers,
Gordon

On Mon, May 27, 2019 at 7:52 PM Lasse Nedergaard <[hidden email]> wrote:
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more

 
Reply | Threaded
Open this post in threaded view
|

Re: Restore state class not found exception in 1.8

Lasse Nedergaard
In reply to this post by Tzu-Li (Gordon) Tai
Hi Gordon

We have found a solution but not why it happens on 1.8. 
For it to work we need to call
Env.registertype(Reportmessage.class)

Reportmessage extends ReportmessageBase and the state operator use ReportmessageBase. 
So we need to register all the class’s that extends a class used in state. Don’t know why this is needed in 1.8

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 28. maj 2019 kl. 10.06 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

Did you move the class to a different namespace / package or changed to be a nested class, across the Flink versions?
That would be the only cause I could reason about at the moment.

If possible, could you also have a very minimal snippet / instructions on how I can maybe reproduce this?
That might give me more insight.

Cheers,
Gordon

On Mon, May 27, 2019 at 7:52 PM Lasse Nedergaard <[hidden email]> wrote:
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more

 
Reply | Threaded
Open this post in threaded view
|

Re: Restore state class not found exception in 1.8

Tzu-Li (Gordon) Tai
Hi Lasse,

This is indeed a bit odd. I'll need to reproduce this locally before I can figure out the root problem. Please bear with me for a while, will get back to you on this.

Meanwhile, you mentioned that you only had some jobs failing with the posted exception. Did you figure out any more details on why this was only partially happening?

Cheers,
Gordon

On Tue, May 28, 2019 at 8:59 PM Lasse Nedergaard <[hidden email]> wrote:
Hi Gordon

We have found a solution but not why it happens on 1.8. 
For it to work we need to call
Env.registertype(Reportmessage.class)

Reportmessage extends ReportmessageBase and the state operator use ReportmessageBase. 
So we need to register all the class’s that extends a class used in state. Don’t know why this is needed in 1.8

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 28. maj 2019 kl. 10.06 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

Did you move the class to a different namespace / package or changed to be a nested class, across the Flink versions?
That would be the only cause I could reason about at the moment.

If possible, could you also have a very minimal snippet / instructions on how I can maybe reproduce this?
That might give me more insight.

Cheers,
Gordon

On Mon, May 27, 2019 at 7:52 PM Lasse Nedergaard <[hidden email]> wrote:
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more

 
Reply | Threaded
Open this post in threaded view
|

Re: Restore state class not found exception in 1.8

Lasse Nedergaard
Hi Gordon 

To us it looks like the env.registerclass is needed when we write the save point. If we have an existing save point without the classes registered it doesn’t work. 

We have only seen the exception in our own sink that store pending data in operator state through CheckpointedFunction interface and this sink isn’t used in all our jobs. 

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 3. jun. 2019 kl. 12.50 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

This is indeed a bit odd. I'll need to reproduce this locally before I can figure out the root problem. Please bear with me for a while, will get back to you on this.

Meanwhile, you mentioned that you only had some jobs failing with the posted exception. Did you figure out any more details on why this was only partially happening?

Cheers,
Gordon

On Tue, May 28, 2019 at 8:59 PM Lasse Nedergaard <[hidden email]> wrote:
Hi Gordon

We have found a solution but not why it happens on 1.8. 
For it to work we need to call
Env.registertype(Reportmessage.class)

Reportmessage extends ReportmessageBase and the state operator use ReportmessageBase. 
So we need to register all the class’s that extends a class used in state. Don’t know why this is needed in 1.8

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 28. maj 2019 kl. 10.06 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

Did you move the class to a different namespace / package or changed to be a nested class, across the Flink versions?
That would be the only cause I could reason about at the moment.

If possible, could you also have a very minimal snippet / instructions on how I can maybe reproduce this?
That might give me more insight.

Cheers,
Gordon

On Mon, May 27, 2019 at 7:52 PM Lasse Nedergaard <[hidden email]> wrote:
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more

 
Reply | Threaded
Open this post in threaded view
|

Re: Restore state class not found exception in 1.8

Tzu-Li (Gordon) Tai
Hi Lasse,

I think the diagnosis here: https://issues.apache.org/jira/browse/FLINK-13159 matches your problem.
This problem should be fixed in the next bugfix version for 1.8.x. We'll also try to fix this for the upcoming 1.9.0 as well.

Cheers,
Gordon

On Mon, Jun 3, 2019 at 1:55 PM Lasse Nedergaard <[hidden email]> wrote:
Hi Gordon 

To us it looks like the env.registerclass is needed when we write the save point. If we have an existing save point without the classes registered it doesn’t work. 

We have only seen the exception in our own sink that store pending data in operator state through CheckpointedFunction interface and this sink isn’t used in all our jobs. 

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 3. jun. 2019 kl. 12.50 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

This is indeed a bit odd. I'll need to reproduce this locally before I can figure out the root problem. Please bear with me for a while, will get back to you on this.

Meanwhile, you mentioned that you only had some jobs failing with the posted exception. Did you figure out any more details on why this was only partially happening?

Cheers,
Gordon

On Tue, May 28, 2019 at 8:59 PM Lasse Nedergaard <[hidden email]> wrote:
Hi Gordon

We have found a solution but not why it happens on 1.8. 
For it to work we need to call
Env.registertype(Reportmessage.class)

Reportmessage extends ReportmessageBase and the state operator use ReportmessageBase. 
So we need to register all the class’s that extends a class used in state. Don’t know why this is needed in 1.8

Med venlig hilsen / Best regards
Lasse Nedergaard


Den 28. maj 2019 kl. 10.06 skrev Tzu-Li (Gordon) Tai <[hidden email]>:

Hi Lasse,

Did you move the class to a different namespace / package or changed to be a nested class, across the Flink versions?
That would be the only cause I could reason about at the moment.

If possible, could you also have a very minimal snippet / instructions on how I can maybe reproduce this?
That might give me more insight.

Cheers,
Gordon

On Mon, May 27, 2019 at 7:52 PM Lasse Nedergaard <[hidden email]> wrote:
Hi.

When we restart some of our jobs from a savepoint we see the the exception below. It only happens for some of our jobs and we didn't see it in 1.7.2. The class Flink can't find differ from job to job and we are sure it's included in our Fat jar.
As a side note we are on our way to use Avro instead of POJO, but are not there yet.
If anyone have a clue what the root cause could be, and how to resolve it would be appreciated.
Thanks in advance

Lasse Nedergaard

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for StreamSink_609b5f7fc746f29234b038c121356a9b_(2/2) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Failed when trying to restore operator state backend
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:86)
	at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createOperatorStateBackend(RocksDBStateBackend.java:537)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$operatorStateBackend$0(StreamTaskStateInitializerImpl.java:246)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
	... 7 more
Caused by: java.lang.RuntimeException: Cannot instantiate class.
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:384)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.deserializeOperatorStateValues(OperatorStateRestoreOperation.java:191)
	at org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:165)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder.build(DefaultOperatorStateBackendBuilder.java:83)
	... 11 more
Caused by: java.lang.ClassNotFoundException: org/trackunit/tm2/formats/ReportMessage
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:382)
	... 14 more