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

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

benlamonica opened a new pull request #6610: [FLINK-10204] - fix 
serialization/copy error for LatencyMarker records.
URL: https://github.com/apache/flink/pull/6610
 
 
   *Thank you very much for contributing to Apache Flink - we are happy that 
you want to help us improve Flink. To help the community review your 
contribution in the best possible way, please go through the checklist below, 
which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a 
hassle. In order to uphold a high standard of quality for code contributions, 
while at the same time managing a large number of contributions, we need 
contributors to prepare the contributions well, and give reviewers enough 
contextual information for the review. Please also understand that 
contributions that do not follow this guide will take longer to review and thus 
typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA 
issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are 
made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the 
pull request", where *FLINK-XXXX* should be replaced by the actual issue 
number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following 
this pattern: `[hotfix] [docs] Fix typo in event time introduction` or 
`[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the 
pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean 
verify` passes. You can set up Travis CI to do that following [this 
guide](http://flink.apache.org/contribute-code.html#best-practices).
   
     - Each pull request should address only one issue, not mix up code from 
multiple issues.
     
     - Each commit in the pull request has a meaningful commit message 
(including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and 
this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   
   ## What is the purpose of the change
   
   Fixes a serialization error with regards to the LatencyMarker not having the 
same binary format when serializing, and when copying the byte stream.  The 
following exception occurs after a long running flink job has been running for 
a while:
   
   `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)`
   
   ## Brief change log
     - *Added LatencyMarker to the StreamElementSerializerTest and then fixed 
the failure by correcting the StreamElementSerializer::copy method.
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
    - * Commenting out the fix in the class will show that when serializing and 
deserializing, you will encounter an EOFException due to the incorrect copy*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: yes (modified the equals/hashCode on the LatencyMarker 
class so that the test would pass)
     - The serializers: yes
     - The runtime per-record code paths (performance sensitive): don't know
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: don't know
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable

----------------------------------------------------------------
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:
us...@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)

Reply via email to