Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-7157

Connect TimestampConverter SMT doesn't handle null values

Agile BoardAttach filesAttach ScreenshotVotersStop watchingWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    Description

      TimestampConverter SMT is not able to handle null values (in any versions), so it's always trying to apply the transformation to the value. Instead, it needs to check for null and use the default value for the new schema's field.

      [2018-07-03 02:31:52,490] ERROR Task MySourceConnector-2 threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask) 
      java.lang.NullPointerException 
      at org.apache.kafka.connect.transforms.TimestampConverter$2.toRaw(TimestampConverter.java:137) 
      at org.apache.kafka.connect.transforms.TimestampConverter.convertTimestamp(TimestampConverter.java:440) 
      at org.apache.kafka.connect.transforms.TimestampConverter.applyValueWithSchema(TimestampConverter.java:368) 
      at org.apache.kafka.connect.transforms.TimestampConverter.applyWithSchema(TimestampConverter.java:358) 
      at org.apache.kafka.connect.transforms.TimestampConverter.apply(TimestampConverter.java:275) 
      at org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) 
      at org.apache.kafka.connect.runtime.WorkerSinkTask.convertMessages(WorkerSinkTask.java:435) 
      at org.apache.kafka.connect.runtime.WorkerSinkTask.poll(WorkerSinkTask.java:264) 
      at org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:182) 
      at org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:150) 
      at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:146) 
      at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:190) 
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
      at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
      at java.lang.Thread.run(Thread.java:748) 
      [2018-07-03 02:31:52,491] ERROR Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask) 
      

      Attachments

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            nimfadora Valeria Vasylieva
            rhauch Randall Hauch
            Votes:
            4 Vote for this issue
            Watchers:
            8 Stop watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment