I'm afraid it's not being printed out due to different log levels :(

Yes, I build the image myself. It takes the tar file from
https://archive.apache.org/dist/flink/flink-1.9.0/
<https://archive.apache.org/dist/flink/flink-1.9.1/> and unpacks it into
the image.
I've ran:

find . -iname "*.jar" | xargs -n 1 jar tf | grep -i producerrecord
find . -iname "*.jar" | xargs -n 1 jar tf | grep -i kafka

Both from within /lib, they both produce no results.

On Tue, Aug 25, 2020 at 10:07 PM Chesnay Schepler <ches...@apache.org>
wrote:

> The NoSuchMethodException shows that the class is still on the classpath,
> but with a different version than your code is expecting. Otherwise you
> would've gotten a different error.
> This implies that there are 2 versions of the kafka dependencies on the
> classpath in your original run; it suddenly working with parent-first
> classloading reinforces the suspicion that they are present in the
> distribution.
>
> As Arvid mentioned, the classpath log entry (at the very start of the log
> file) would be interesting.
>
> Did you build the Flink yourself distribution, or are you relying on one
> of the existing Flink binaries/images?
>
> On 25/08/2020 20:51, Yuval Itzchakov wrote:
>
> Hi Arvid,
> I'm running Flink in a job cluster on k8s using the Lyft Operator.
>
> The flink image that I'm building does not have the flink-connector-kafka
> library in it's JAR, I've made sure of this using `jar -tf`. Additionally,
> once I removed the dependency from my uber jar, it failed with a
> "NoSuchMethodException" at runtime for one of the arbitrary methods.
>
> I used classloader.resolve-order: parent-first and it resolved the issue
> somehow. I still don't know why though.
>
> On Tue, Aug 25, 2020 at 6:13 PM Arvid Heise <ar...@ververica.com> wrote:
>
>> Hi Yuval,
>>
>> How do you execute Flink? Can you show us the log entry with the
>> classpath?
>>
>> I'm guessing that you have Kafka bundled in your uber-jar and
>> additionally also have the connector in flink-dist/lib. If so, you simply
>> need to remove it in one place. In general, if you use flink-dist, you'd
>> not bundle any Flink dependencies in your uber-jar (use provided scope for
>> them).
>>
>> If you have everything bundled in one uber-jar and execute it somehow
>> without flink-dist, then I don't immediately see a solution. Then the log
>> with the classpath would help.
>>
>> Best,
>>
>> Arvid
>>
>>
>> On Sun, Aug 23, 2020 at 1:37 PM Yuval Itzchakov <yuva...@gmail.com>
>> wrote:
>>
>>> Hi,
>>> I'm trying to load a FlinkKafkaProducer sink alongside another custom
>>> sink. While trying to restore
>>> a running Flink app from the previous state, I get the error message
>>> below.
>>> I am running Flink 1.9.0 with the following SBT dependency added:
>>> "org.apache.flink" %% "flink-connector-kafka" % 1.9.0
>>> And the app is deployed via a standard uber jar with all the
>>> dependencies. W
>>> Would appreciate the help
>>> java.lang.LinkageError: loader constraint violation: loader (instance of
>>> org/apache/flink/util/ChildFirstClassLoader) previously initiated
>>> loading for a different type with name
>>> "org/apache/kafka/clients/producer/ProducerRecord"
>>>     at java.lang.ClassLoader.defineClass1(Native Method)
>>>     at java.lang.ClassLoader.defineClass(ClassLoader.java:763)
>>>     at java.security.SecureClassLoader.defineClass(SecureClassLoader
>>> .java:142)
>>>     at java.net.URLClassLoader.defineClass(URLClassLoader.java:468)
>>>     at java.net.URLClassLoader.access$100(URLClassLoader.java:74)
>>>     at java.net.URLClassLoader$1.run(URLClassLoader.java:369)
>>>     at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
>>>     at java.security.AccessController.doPrivileged(Native Method)
>>>     at java.net.URLClassLoader.findClass(URLClassLoader.java:362)
>>>     at org.apache.flink.util.ChildFirstClassLoader.loadClass(
>>> ChildFirstClassLoader.java:66)
>>>     at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>>     at java.lang.Class.getDeclaredMethods0(Native Method)
>>>     at java.lang.Class.privateGetDeclaredMethods(Class.java:2701)
>>>     at java.lang.Class.getDeclaredMethod(Class.java:2128)
>>>     at java.io.ObjectStreamClass.getPrivateMethod(ObjectStreamClass
>>> .java:1629)
>>>     at java.io.ObjectStreamClass.access$1700(ObjectStreamClass.java:79)
>>>     at java.io.ObjectStreamClass$3.run(ObjectStreamClass.java:520)
>>>     at java.io.ObjectStreamClass$3.run(ObjectStreamClass.java:494)
>>>     at java.security.AccessController.doPrivileged(Native Method)
>>>     at java.io.ObjectStreamClass.<init>(ObjectStreamClass.java:494)
>>>     at java.io.ObjectStreamClass.lookup(ObjectStreamClass.java:391)
>>>     at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:681
>>> )
>>>     at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream
>>> .java:1885)
>>>     at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:
>>> 1751)
>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>> .java:2042)
>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573
>>> )
>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>> .java:2287)
>>>     at java.io.ObjectInputStream.defaultReadObject(ObjectInputStream
>>> .java:561)
>>>     at org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer
>>> .readObject(FlinkKafkaProducer.java:1202)
>>>     at sun.reflect.GeneratedMethodAccessor358.invoke(Unknown Source)
>>>     at sun.reflect.DelegatingMethodAccessorImpl.invoke(
>>> DelegatingMethodAccessorImpl.java:43)
>>>     at java.lang.reflect.Method.invoke(Method.java:498)
>>>     at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass
>>> .java:1170)
>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>> 2178)
>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>> .java:2069)
>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573
>>> )
>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>> .java:2287)
>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>> 2211)
>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>> .java:2069)
>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573
>>> )
>>>     at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream
>>> .java:2287)
>>>     at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:
>>> 2211)
>>>     at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream
>>> .java:2069)
>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573
>>> )
>>>     at java.io.ObjectInputStream.readObject(ObjectInputStream.java:431)
>>>     at org.apache.flink.util.InstantiationUtil.deserializeObject(
>>> InstantiationUtil.java:576)
>>>     at org.apache.flink.util.InstantiationUtil.deserializeObject(
>>> InstantiationUtil.java:562)
>>>     at org.apache.flink.util.InstantiationUtil.deserializeObject(
>>> InstantiationUtil.java:550)
>>>     at org.apache.flink.util.InstantiationUtil.readObjectFromConfig(
>>> InstantiationUtil.java:511)
>>>     at org.apache.flink.streaming.api.graph.StreamConfig
>>> .getStreamOperatorFactory(StreamConfig.java:235)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:427)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createChainedOperator(OperatorChain.java:418)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain
>>> .createOutputCollector(OperatorChain.java:354)
>>>     at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(
>>> OperatorChain.java:144)
>>>     at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(
>>> StreamTask.java:370)
>>>     at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:705)
>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:530)
>>>     at java.lang.Thread.run(Thread.java:748)
>>>
>>> --
>>> Best Regards,
>>> Yuval Itzchakov.
>>>
>>
>>
>> --
>>
>> 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
>>
>>
>
>
> --
> Best Regards,
> Yuval Itzchakov.
>
>
>

-- 
Best Regards,
Yuval Itzchakov.

Reply via email to