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

[CONNECT] It looks like we are not backing off properly when reconfiguring tasks

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.5.0
    • connect
    • None

    Description

      Looking at logs of attempt to reconfigure connector while leader is restarting, I see:

      [2016-01-29 20:31:01,799] ERROR IO error forwarding REST request:  (org.apache.kafka.connect.runtime.rest.RestServer)
      java.net.ConnectException: Connection refused
      [2016-01-29 20:31:01,802] ERROR Request to leader to reconfigure connector tasks failed (org.apache.kafka.connect.runtime.distributed.DistributedHerder)
      org.apache.kafka.connect.runtime.rest.errors.ConnectRestException: IO Error trying to forward REST request: Connection refused
      [2016-01-29 20:31:01,802] ERROR Failed to reconfigure connector's tasks, retrying after backoff: (org.apache.kafka.connect.runtime.distributed.DistributedHerder)
      org.apache.kafka.connect.runtime.rest.errors.ConnectRestException: IO Error trying to forward REST request: Connection refused
      [2016-01-29 20:31:01,803] DEBUG Sending POST with input [{"tables":"bar","table.poll.interval.ms":"1000","incrementing.column.name":"id","connection.url":"jdbc:mysql://worker1:3306/testdb?user=root","name":"test-mysql-jdbc","tasks.max":"3","task.class":"io.confluent.connect.jdbc.JdbcSourceTask","poll.interval.ms":"1000","topic.prefix":"test-","connector.class":"io.confluent.connect.jdbc.JdbcSourceConnector","mode":"incrementing"},{"tables":"foo","table.poll.interval.ms":"1000","incrementing.column.name":"id","connection.url":"jdbc:mysql://worker1:3306/testdb?user=root","name":"test-mysql-jdbc","tasks.max":"3","task.class":"io.confluent.connect.jdbc.JdbcSourceTask","poll.interval.ms":"1000","topic.prefix":"test-","connector.class":"io.confluent.connect.jdbc.JdbcSourceConnector","mode":"incrementing"}] to http://worker2:8083/connectors/test-mysql-jdbc/tasks (org.apache.kafka.connect.runtime.rest.RestServer)
      [2016-01-29 20:31:01,803] ERROR IO error forwarding REST request:  (org.apache.kafka.connect.runtime.rest.RestServer)
      java.net.ConnectException: Connection refused
      [2016-01-29 20:31:01,804] ERROR Request to leader to reconfigure connector tasks failed (org.apache.kafka.connect.runtime.distributed.DistributedHerder)
      org.apache.kafka.connect.runtime.rest.errors.ConnectRestException: IO Error trying to forward REST request: Connection refused
      

      Note that it looks like we are retrying every 1ms, while I'd expect a retry every 250ms.

      Attachments

        Issue Links

          Activity

            People

              liquanpei Liquan Pei
              gwenshap Gwen Shapira
              Votes:
              1 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: