[ 
https://issues.apache.org/jira/browse/FLINK-34497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated FLINK-34497:
-----------------------------------
    Labels: pull-request-available  (was: )

> Avoid using system classloader in SerializedThrowableDeserializer  
> -------------------------------------------------------------------
>
>                 Key: FLINK-34497
>                 URL: https://issues.apache.org/jira/browse/FLINK-34497
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / REST
>    Affects Versions: 1.17.2, 1.18.1
>            Reporter: jrthe42
>            Priority: Major
>              Labels: pull-request-available
>
> SerializedThrowableDeserializer is now using 
> `ClassLoader.getSystemClassLoader()` when deserializing 
> `SerializedThrowable`. But when using flink-client in systems like spring 
> boot, we will get exceptions like this:
> {code:java}
> java.lang.ClassNotFoundException: org.apache.flink.util.SerializedThrowable
>       at 
> java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641)
>       at 
> java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188)
>       at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525)
>       at java.base/java.lang.Class.forName0(Native Method)
>       at java.base/java.lang.Class.forName(Class.java:467)
>       at 
> org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.resolveClass(InstantiationUtil.java:78)
>       at 
> java.base/java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:2034)
>       at 
> java.base/java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1898)
>       at 
> java.base/java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2224)
>       at 
> java.base/java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1733)
>       at 
> java.base/java.io.ObjectInputStream.readObject(ObjectInputStream.java:509)
>       at 
> java.base/java.io.ObjectInputStream.readObject(ObjectInputStream.java:467)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:539)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:527)
>       at 
> org.apache.flink.runtime.rest.messages.json.SerializedThrowableDeserializer.deserialize(SerializedThrowableDeserializer.java:69)
>       at 
> org.apache.flink.runtime.rest.messages.json.JobResultDeserializer.deserialize(JobResultDeserializer.java:106)
>       at 
> org.apache.flink.runtime.rest.messages.json.JobResultDeserializer.deserialize(JobResultDeserializer.java:50)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.SettableBeanProperty.deserialize(SettableBeanProperty.java:542)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeWithErrorWrapping(BeanDeserializer.java:564)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:439)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1405)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserializeFromObject(BeanDeserializer.java:352)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserialize(BeanDeserializer.java:185)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.DefaultDeserializationContext.readRootValue(DefaultDeserializationContext.java:323)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper._readValue(ObjectMapper.java:4706)
>       at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:2948)
>       at 
> org.apache.flink.runtime.rest.RestClient.parseResponse(RestClient.java:635)
>       at 
> org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$6(RestClient.java:626)
>       at 
> java.base/java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1150)
>       at 
> java.base/java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:482)
>       at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
>       at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>       at java.base/java.lang.Thread.run(Thread.java:840)
> {code}
> This is because Spring Boot is using [The Executable Jar 
> Format,|https://docs.spring.io/spring-boot/docs/current/reference/html/executable-jar.html#appendix.executable-jar.restrictions]
>  which contains Nested JARs, the system classloader is not able to load the 
> class in the nested jar, thus will lead to class not found exception. We 
> should use current context classloader instead.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to