You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Gregory Fee (JIRA)" <ji...@apache.org> on 2018/07/20 19:53:00 UTC

[jira] [Created] (FLINK-9905) ClassCastException: org.apache.flink.streaming.runtime.streamrecord.LatencyMarker

Gregory Fee created FLINK-9905:
----------------------------------

             Summary: ClassCastException: org.apache.flink.streaming.runtime.streamrecord.LatencyMarker
                 Key: FLINK-9905
                 URL: https://issues.apache.org/jira/browse/FLINK-9905
             Project: Flink
          Issue Type: Bug
          Components: Core
    Affects Versions: 1.4.2
            Reporter: Gregory Fee


The example program feeds from a couple of data sources into a SQL transformation which then sinks out via a GRPC call. I'm not sure what context is relevant but I can provide additional context as necessary. The stack trace below is what is reported in the Flink UI as the exception.

 

org.apache.flink.streaming.runtime.tasks.ExceptionInChainedOperatorException: Could not forward element to next operator

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:566)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:524)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:504)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:830)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:808)

at com.lyft.streamingplatform.BetterBuffer$OutputThread.run(BetterBuffer.java:316)

Caused by: org.apache.flink.streaming.runtime.tasks.ExceptionInChainedOperatorException: Could not forward element to next operator

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:566)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:524)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:504)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:830)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:808)

at org.apache.flink.streaming.api.operators.TimestampedCollector.collect(TimestampedCollector.java:51)

at com.lyft.dsp.functions.Labeler$UnlabelerFunction.processElement(Labeler.java:67)

at com.lyft.dsp.functions.Labeler$UnlabelerFunction.processElement(Labeler.java:48)

at org.apache.flink.streaming.api.operators.ProcessOperator.processElement(ProcessOperator.java:66)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:549)

... 5 more

Caused by: org.apache.flink.streaming.runtime.tasks.ExceptionInChainedOperatorException: Could not forward element to next operator

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:566)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:524)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:504)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:830)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:808)

at org.apache.flink.streaming.api.operators.TimestampedCollector.collect(TimestampedCollector.java:51)

at org.apache.flink.table.runtime.CRowWrappingCollector.collect(CRowWrappingCollector.scala:37)

at org.apache.flink.table.runtime.CRowWrappingCollector.collect(CRowWrappingCollector.scala:28)

at DataStreamSourceConversion$14.processElement(Unknown Source)

at org.apache.flink.table.runtime.CRowOutputProcessRunner.processElement(CRowOutputProcessRunner.scala:67)

at org.apache.flink.streaming.api.operators.ProcessOperator.processElement(ProcessOperator.java:66)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:549)

... 14 more

Caused by: java.lang.RuntimeException: org.apache.flink.streaming.runtime.streamrecord.LatencyMarker cannot be cast to org.apache.flink.streaming.runtime.streamrecord.StreamRecord

at org.apache.flink.streaming.runtime.io.RecordWriterOutput.pushToRecordWriter(RecordWriterOutput.java:105)

at org.apache.flink.streaming.runtime.io.RecordWriterOutput.collect(RecordWriterOutput.java:84)

at org.apache.flink.streaming.runtime.io.RecordWriterOutput.collect(RecordWriterOutput.java:42)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:830)

at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:808)

at org.apache.flink.streaming.api.operators.TimestampedCollector.collect(TimestampedCollector.java:51)

at org.apache.flink.table.runtime.CRowWrappingCollector.collect(CRowWrappingCollector.scala:37)

at org.apache.flink.table.runtime.CRowWrappingCollector.collect(CRowWrappingCollector.scala:28)

at DataStreamCalcRule$37.processElement(Unknown Source)

at org.apache.flink.table.runtime.CRowProcessRunner.processElement(CRowProcessRunner.scala:66)

at org.apache.flink.table.runtime.CRowProcessRunner.processElement(CRowProcessRunner.scala:35)

at org.apache.flink.streaming.api.operators.ProcessOperator.processElement(ProcessOperator.java:66)

at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:549)

... 25 more

Caused by: java.lang.ClassCastException: org.apache.flink.streaming.runtime.streamrecord.LatencyMarker cannot be cast to org.apache.flink.streaming.runtime.streamrecord.StreamRecord

at org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner.selectChannels(KeyGroupStreamPartitioner.java:61)

at org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner.selectChannels(KeyGroupStreamPartitioner.java:32)

at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:88)

at org.apache.flink.streaming.runtime.io.StreamRecordWriter.emit(StreamRecordWriter.java:84)

at org.apache.flink.streaming.runtime.io.RecordWriterOutput.pushToRecordWriter(RecordWriterOutput.java:102)

... 37 more



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