flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-10204) Job is marked as FAILED after serialization exception
Date Fri, 24 Aug 2018 14:08:00 GMT

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

ASF GitHub Bot commented on FLINK-10204:
----------------------------------------

benlamonica commented on issue #6610: [FLINK-10204] - fix serialization/copy error for LatencyMarker
records.
URL: https://github.com/apache/flink/pull/6610#issuecomment-415769746
 
 
   @StephanEwen and @zentol , I've commented on the PR comments. The reason why LatencyMarker
has changes is because the equals implementation was incorrect. It did an operatorId != that.operatorId,
instead of !operatorId.equals(that.operatorId).
   
   If you want, I can revert that change and just write a more complicated assertEquals statement
in the test, but I feel that it is the correct change. The other stream records have equals
correctly implemented so that they can compare after a copy.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Job is marked as FAILED after serialization exception
> -----------------------------------------------------
>
>                 Key: FLINK-10204
>                 URL: https://issues.apache.org/jira/browse/FLINK-10204
>             Project: Flink
>          Issue Type: Bug
>            Reporter: Ben La Monica
>            Priority: Major
>              Labels: pull-request-available
>
> We have a long running flink job that eventually fails and is shut down due to an internal
serialization exception that we keep on getting. Here is the stack trace:
> {code:java}
> 2018-08-23 18:39:48,199 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph -
Job NAV Estimation (4b5463d76167f9f5aac83a890e8a867d) switched from state FAILING to FAILED.
> java.io.IOException: Corrupt stream, found tag: 127
> at org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer.deserialize(StreamElementSerializer.java:219)
> at org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer.deserialize(StreamElementSerializer.java:49)
> at org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate.read(NonReusingDeserializationDelegate.java:55)
> at org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer.getNextRecord(SpillingAdaptiveSpanningRecordDeserializer.java:140)
> at org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor.processInput(StreamTwoInputProcessor.java:206)
> at org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask.run(TwoInputStreamTask.java:115)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:306)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703)
> at java.lang.Thread.run(Thread.java:748){code}
>  
> I think I have tracked down the issue to a mismatch in the serialization/deserialization/copy
code in the StreamElementSerializer with regards to the LATENCY_MARKER.
> The Serialization logic writes 3 LONGs and an INT. The copy logic only writes (and reads)
a LONG and 2 INTs. Adding a test for the LatencyMarker throws an EOFException, and fixing
the copy code causes the test to pass again.
> I've written a unit test that highlights the problem, and have written the code to correct
it.
> I'll submit a PR that goes along with it.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message