[ 
https://issues.apache.org/jira/browse/FLINK-16794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17068481#comment-17068481
 ] 

victor.jiang commented on FLINK-16794:
--------------------------------------

java.lang.RuntimeException: StreamJobMgmtClient-StreamJobMgmtFacade kpstats 
running failed:java.lang.RuntimeException: 
StreamJobMgmtClient-StreamJobMgmtFacade kpstats running failed: at 
com.bluecore.core.streaming.api.StreamJobMgmtClient.runJob(StreamJobMgmtClient.java:77)
 at 
com.bluecore.core.streaming.api.StreamJobMgmtClient.main(StreamJobMgmtClient.java:100)Caused
 by: java.lang.RuntimeException: The stream[kpstats] executing 
failed:org.apache.flink.client.program.ProgramInvocationException: Could not 
retrieve the execution result. (JobID: 2fbb2558e4eb034f60bec101da4ad664) at 
com.bluecore.core.streaming.spi.FlinkStreamJobMgmtFacadeImpl.run(FlinkStreamJobMgmtFacadeImpl.java:176)
 at 
com.bluecore.core.streaming.api.StreamJobMgmtClient.runJob(StreamJobMgmtClient.java:73)
 ... 1 moreCaused by: 
org.apache.flink.client.program.ProgramInvocationException: Could not retrieve 
the execution result. (JobID: 2fbb2558e4eb034f60bec101da4ad664) at 
org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:261)
 at org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:483) 
at 
com.bluecore.core.streaming.spi.FlinkStreamContextEnvironment.execute(FlinkStreamContextEnvironment.java:39)
 at 
com.bluecore.viva.streaming.misc.KPartitionStats.execute(KPartitionStats.java:171)
 at 
com.bluecore.core.streaming.spi.FlinkStreamJobMgmtFacadeImpl.run(FlinkStreamJobMgmtFacadeImpl.java:156)
 ... 2 moreCaused by: 
org.apache.flink.runtime.concurrent.FutureUtils$RetryException: Could not 
complete the operation. Number of retries has been exhausted. at 
org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$5(FutureUtils.java:218)
 at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
 at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
 at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) 
at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:561) 
at 
java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:929)
 at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
at java.lang.Thread.run(Thread.java:748)Caused by: 
java.util.concurrent.CompletionException: 
org.apache.flink.runtime.rest.util.RestClientException: Response was neither of 
the expected type([simple type, class 
org.apache.flink.runtime.rest.messages.job.JobExecutionResultResponseBody]) nor 
an error. at 
java.util.concurrent.CompletableFuture.encodeRelay(CompletableFuture.java:326) 
at 
java.util.concurrent.CompletableFuture.completeRelay(CompletableFuture.java:338)
 at java.util.concurrent.CompletableFuture.uniRelay(CompletableFuture.java:911) 
at 
java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:953) 
at 
java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926)
 ... 4 moreCaused by: org.apache.flink.runtime.rest.util.RestClientException: 
Response was neither of the expected type([simple type, class 
org.apache.flink.runtime.rest.messages.job.JobExecutionResultResponseBody]) nor 
an error. at 
org.apache.flink.runtime.rest.RestClient.parseResponse(RestClient.java:398) at 
org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$3(RestClient.java:373)
 at 
java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:952) 
... 5 moreCaused by: java.io.IOException: Failed to deserialize 
org.apache.flink.util.SerializedThrowable at 
org.apache.flink.runtime.rest.messages.json.SerializedThrowableDeserializer.deserialize(SerializedThrowableDeserializer.java:54)
 at 
org.apache.flink.runtime.rest.messages.json.JobResultDeserializer.deserialize(JobResultDeserializer.java:104)
 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:530)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeWithErrorWrapping(BeanDeserializer.java:528)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:417)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1287)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserializeFromObject(BeanDeserializer.java:326)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserialize(BeanDeserializer.java:159)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper._readValue(ObjectMapper.java:4173)
 at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:2536)
 at org.apache.flink.runtime.rest.RestClient.parseResponse(RestClient.java:381) 
... 7 moreCaused by: java.lang.ClassNotFoundException: 
org.apache.flink.util.SerializedThrowable at 
java.net.URLClassLoader.findClass(URLClassLoader.java:381) at 
java.lang.ClassLoader.loadClass(ClassLoader.java:424) at 
sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:338) at 
java.lang.ClassLoader.loadClass(ClassLoader.java:357) at 
java.lang.Class.forName0(Native Method) at 
java.lang.Class.forName(Class.java:348) at 
org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.resolveClass(InstantiationUtil.java:78)
 at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1863) at 
java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1746) at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2037) at 
java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1568) at 
java.io.ObjectInputStream.readObject(ObjectInputStream.java:428) at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:566)
 at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:552)
 at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:540)
 at 
org.apache.flink.runtime.rest.messages.json.SerializedThrowableDeserializer.deserialize(SerializedThrowableDeserializer.java:52)
 ... 18 more

> ClassNotFoundException caused by ClassLoader.getSystemClassLoader using 
> impertinently  
> ---------------------------------------------------------------------------------------
>
>                 Key: FLINK-16794
>                 URL: https://issues.apache.org/jira/browse/FLINK-16794
>             Project: Flink
>          Issue Type: Improvement
>          Components: Client / Job Submission, Runtime / REST
>    Affects Versions: 1.8.0, 1.8.1, 1.8.2, 1.8.3
>            Reporter: victor.jiang
>            Priority: Major
>
> In some containerization environment,the context classloader is not the 
> SystemClassLoader,it uses the customized classloader usually for the classes 
> isolation ,so the ClassNotFoundException may be caused。recommends using 
> getClass/Caller/ThreadCurrentContext 's ClassLoader。
> The related sources below:
> 1.flink-clients\src\main\java\org\apache\flink\client\program\ClusterClient.java"(690,33):
>  return getAccumulators(jobID, ClassLoader.getSystemClassLoader());
>  
> 2.flink-clients\src\main\java\org\apache\flink\client\program\MiniClusterClient.java"(148,33):
>  return getAccumulators(jobID, ClassLoader.getSystemClassLoader());
>  
> 3.flink-runtime\src\main\java\org\apache\flink\runtime\blob\BlobUtils.java"(348,66):
>  return (Throwable) InstantiationUtil.deserializeObject(bytes, 
> ClassLoader.getSystemClassLoader());
>  
> 4.flink-runtime\src\main\java\org\apache\flink\runtime\rest\messages\json\SerializedThrowableDeserializer.java"(52,68):
>  return InstantiationUtil.deserializeObject(serializedException, 
> ClassLoader.getSystemClassLoader());
>  
> 5.flink-runtime\src\main\java\org\apache\flink\runtime\rpc\messages\RemoteRpcInvocation.java"(118,67):
>  methodInvocation = 
> serializedMethodInvocation.deserializeValue(ClassLoader.getSystemClassLoader());



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to