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

Thomas Weise commented on FLINK-38148:
--------------------------------------

We run into this also. Although it is technically correct that 1.17 now is 4 
versions away, that includes 2.x and we probably could be more lenient with the 
version support. In a typical infra there could always be stragglers and those 
could then hold back operator upgrades, if the older Flink version is no longer 
supported.

> Failed serialization of JobExceptionsInfoWithHistory in 
> flink-kubernetes-operator
> ---------------------------------------------------------------------------------
>
>                 Key: FLINK-38148
>                 URL: https://issues.apache.org/jira/browse/FLINK-38148
>             Project: Flink
>          Issue Type: Bug
>          Components: Kubernetes Operator, Runtime / REST
>    Affects Versions: 1.16.0, 1.17.0, kubernetes-operator-1.12.1
>            Reporter: Michał Fijołek
>            Assignee: Michał Fijołek
>            Priority: Major
>
> Hi team. We have just upgraded flink-k8s-operator to 1.12.1 and discovered 
> that for flink jobs running v.1_17 or earlier, there are such error logs in 
> operator:
>  
> {noformat}
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.exc.ValueInstantiationException:
>  Cannot construct instance of 
> `org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory$RootExceptionInfo`,
>  problem: `java.lang.NullPointerException`
>  at [Source: UNKNOWN; byte offset: #UNKNOWN[] (through reference chain: 
> org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory["exceptionHistory"]->org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory$JobExceptionHistory["entries"]->java.util.ArrayList[0])
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.exc.ValueInstantiationException.from(ValueInstantiationException.java:47)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext.instantiationException(DeserializationContext.java:2052)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdValueInstantiator.wrapAsJsonMappingException(StdValueInstantiator.java:587)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdValueInstantiator.rewrapCtorProblem(StdValueInstantiator.java:610)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdValueInstantiator.createFromObjectWith(StdValueInstantiator.java:293)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.ValueInstantiator.createFromObjectWith(ValueInstantiator.java:288)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.impl.PropertyBasedCreator.build(PropertyBasedCreator.java:202)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:519)
>   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.std.CollectionDeserializer._deserializeFromArray(CollectionDeserializer.java:359)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer.deserialize(CollectionDeserializer.java:244)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer.deserialize(CollectionDeserializer.java:28)
>   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.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:646)
>   at 
> org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$6(RestClient.java:628)
>   at 
> java.base/java.util.concurrent.CompletableFuture$UniCompose.tryFire(Unknown 
> Source)
>   at java.base/java.util.concurrent.CompletableFuture$Completion.run(Unknown 
> Source)
>   at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown 
> Source)
>   at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown 
> Source)
>   at java.base/java.lang.Thread.run(Unknown Source)
> Caused by: java.lang.NullPointerException
>   at org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:59)
>   at 
> org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory$ExceptionInfo.<init>(JobExceptionsInfoWithHistory.java:199)
>   at 
> org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory$RootExceptionInfo.<init>(JobExceptionsInfoWithHistory.java:320)
>   at jdk.internal.reflect.GeneratedConstructorAccessor108.newInstance(Unknown 
> Source)
>   at 
> java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(Unknown
>  Source)
>   at java.base/java.lang.reflect.Constructor.newInstance(Unknown Source)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.introspect.AnnotatedConstructor.call(AnnotatedConstructor.java:128)
>   at 
> org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdValueInstantiator.createFromObjectWith(StdValueInstantiator.java:291)
>   ... 31 more{noformat}
> for jobs running on flink v1.18 and above, I don't see such errors.
> To reproduce: apply FlinkDeployment to k8s, wait for job to run, then kill 
> task manager pod manually. In UI you can see job exception in exception 
> history. Operator will start to produce errors in a loop.
> I'm not sure what are negative consequences of these errors.
> The same issue was raised in mailing list: 
> [https://lists.apache.org/thread/xr4wfcn4xxj3ymz6rr7ymcczldn6gp9y]
> I'm happy to help with the implementation
> thank you 🙏



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

Reply via email to