Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-10204

StreamElementSerializer#copy broken for LatencyMarkers

    XMLWordPrintableJSON

Details

    Description

      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:

      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)

       

      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.

      Attachments

        Issue Links

          Activity

            People

              mstar_benl Ben La Monica
              mstar_benl Ben La Monica
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: