You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Konstantin Knauf (Jira)" <ji...@apache.org> on 2020/10/18 16:29:00 UTC

[jira] [Created] (FLINK-19695) Writing Table with RowTime Column of type TIMESTAMP(3) to Kafka fails with ClassCastException

Konstantin Knauf created FLINK-19695:
----------------------------------------

             Summary: Writing Table with RowTime Column of type TIMESTAMP(3) to Kafka fails with ClassCastException
                 Key: FLINK-19695
                 URL: https://issues.apache.org/jira/browse/FLINK-19695
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
    Affects Versions: 1.11.2, 1.12.0
            Reporter: Konstantin Knauf


When I try to write a table to Kafka (JSON format) that has a rowtime attribute of type TIMESTAMP(3) the job fails with 

{noformat}
2020-10-18 18:02:08
java.lang.ClassCastException: org.apache.flink.table.data.TimestampData cannot be cast to java.lang.Long
	at org.apache.flink.table.data.GenericRowData.getLong(GenericRowData.java:154)
	at org.apache.flink.table.runtime.operators.sink.SinkOperator$SimpleContext.timestamp(SinkOperator.java:144)
	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(FlinkKafkaProducer.java:866)
	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(FlinkKafkaProducer.java:99)
	at org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.invoke(TwoPhaseCommitSinkFunction.java:235)
	at org.apache.flink.table.runtime.operators.sink.SinkOperator.processElement(SinkOperator.java:86)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:717)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:692)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:672)
	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:52)
	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:30)
	at org.apache.flink.table.runtime.operators.wmassigners.WatermarkAssignerOperator.processElement(WatermarkAssignerOperator.java:123)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:717)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:692)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:672)
	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:52)
	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:30)
	at org.apache.flink.streaming.api.operators.StreamSourceContexts$NonTimestampContext.collect(StreamSourceContexts.java:104)
	at org.apache.flink.streaming.api.operators.StreamSourceContexts$NonTimestampContext.collectWithTimestamp(StreamSourceContexts.java:111)
	at org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher.emitRecordsWithTimestamps(AbstractFetcher.java:352)
	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.partitionConsumerRecordsHandler(KafkaFetcher.java:185)
	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.runFetchLoop(KafkaFetcher.java:141)
	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755)
	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100)
	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63)
	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:213)
{noformat}

From looking at the relevant code in SinkOperator$SimpleContext#timestamp it seems that we can only deal with long type timestamps in SinkOperator?!




--
This message was sent by Atlassian Jira
(v8.3.4#803005)