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

Writing Table with RowTime Column of type TIMESTAMP(3) to Kafka fails with ClassCastException

    XMLWordPrintableJSON

Details

    Description

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

      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)
      

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

      Attachments

        Issue Links

          Activity

            People

              jark Jark Wu
              knaufk Konstantin Knauf
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: