I just switched from providing my jar while creating a Remote environment
to providing this jar on flink's classpath. It used to work just fine when
the jar was shipped to Fllink with the job graph. Now when jar is available
to flink on the startup the same job that used to run is failing with
exception I provided. I suspect that it might be class loader issue but am
not sure

On Fri, Nov 20, 2020 at 12:10 PM Arvid Heise <ar...@ververica.com> wrote:

> The easiest solution for all non-trivial issues like this is to start the
> application locally in a local executor, so you can debug in your IDE.
>
> Additionally, double-check that you have no lambdas/anonymous classes that
> reference outer classes with ObjectMapper. ObjectMapper should also be
> static as it's fully immutable, so you can also check that.
>
> On Fri, Nov 20, 2020 at 8:55 PM Alexander Bagerman <bager...@gmail.com>
> wrote:
>
>> Thanks, Arvid,
>> That is what I thought too. I went through all the instances where it
>> might 've been a member variable and made sure that it's declared as
>> transient :-( Is there anything else I can check?
>> Alex
>>
>> On Fri, Nov 20, 2020 at 11:50 AM Arvid Heise <ar...@ververica.com> wrote:
>>
>>> Are you using ObjectMapper as a non-transient field? If so, please make
>>> it transient and initialize in open() of a Rich*Function.
>>>
>>> On Fri, Nov 20, 2020 at 7:56 PM Alexander Bagerman <bager...@gmail.com>
>>> wrote:
>>>
>>>> Hi,
>>>> I added my custom jar (that includes dependencies on Jackson) to Flink
>>>> classpath. It seems to be loaded just fine. But when the job starts I am
>>>> getting an exception below. I am sure how to interpret the exception though
>>>> and would appreciate it if somebody gives me advice on it.
>>>> Thanks
>>>> Alex
>>>>
>>>> 2020-11-20 18:34:35,643 WARN org.apache.flink.runtime.taskmanager.Task
>>>> [] - Source: Custom Source -> Flat Map -> Flat Map -> Sink: Unnamed (1/
>>>> 1) (dcbf799dadba5d4b7e7f5af15919a4b6) switched from RUNNING to FAILED.
>>>> org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot
>>>> instantiate user function.
>>>>     at org.apache.flink.streaming.api.graph.StreamConfig
>>>> .getStreamOperatorFactory(StreamConfig.java:275) ~[flink-dist_2.11-1.11
>>>> .2.jar:1.11.2]
>>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>>> .createChainedOperator(OperatorChain.java:471) ~[flink-dist_2.11-1.11.2
>>>> .jar:1.11.2]
>>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>>> .createOutputCollector(OperatorChain.java:393) ~[flink-dist_2.11-1.11.2
>>>> .jar:1.11.2]
>>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(
>>>> OperatorChain.java:155) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.streaming.runtime.tasks.StreamTask
>>>> .beforeInvoke(StreamTask.java:459) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(
>>>> StreamTask.java:528) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
>>>> [flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
>>>> [flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at java.lang.Thread.run(Thread.java:748) [?:1.8.0_265]
>>>> Caused by: java.lang.ClassCastException: cannot assign instance of
>>>> java.util.concurrent.ConcurrentHashMap to field
>>>> com.fasterxml.jackson.databind.deser.DeserializerCache._cachedDeserializers
>>>> of type com.fasterxml.jackson.databind.util.LRUMap in instance of
>>>> com.fasterxml.jackson.databind.deser.DeserializerCache
>>>>     at java.io.ObjectStreamClass$FieldReflector.setObjFieldValues(
>>>> ObjectStreamClass.java:2301) ~[?:1.8.0_265]
>>>>     at java.io.ObjectStreamClass.setObjFieldValues(ObjectStreamClass
>>>> .java:1431) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2372) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2366) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2366) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2366) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2366) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2366) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>>> .java:2366) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>>> 2290) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>>> .java:2148) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:
>>>> 1647) ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject(ObjectInputStream.java:483)
>>>> ~[?:1.8.0_265]
>>>>     at java.io.ObjectInputStream.readObject(ObjectInputStream.java:441)
>>>> ~[?:1.8.0_265]
>>>>     at org.apache.flink.util.InstantiationUtil.deserializeObject(
>>>> InstantiationUtil.java:576) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.util.InstantiationUtil.deserializeObject(
>>>> InstantiationUtil.java:562) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.util.InstantiationUtil.deserializeObject(
>>>> InstantiationUtil.java:550) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.util.InstantiationUtil.readObjectFromConfig(
>>>> InstantiationUtil.java:511) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>>>     at org.apache.flink.streaming.api.graph.StreamConfig
>>>> .getStreamOperatorFactory(StreamConfig.java:260) ~[flink-dist_2.11-1.11
>>>> .2.jar:1.11.2]
>>>>     ... 8 more
>>>>
>>>
>>>
>>> --
>>>
>>> Arvid Heise | Senior Java Developer
>>>
>>> <https://www.ververica.com/>
>>>
>>> Follow us @VervericaData
>>>
>>> --
>>>
>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
>>> Conference
>>>
>>> Stream Processing | Event Driven | Real Time
>>>
>>> --
>>>
>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>>>
>>> --
>>> Ververica GmbH
>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
>>> (Toni) Cheng
>>>
>>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> <https://www.ververica.com/>
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Toni) Cheng
>

Reply via email to