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

System test failure in 0.8.2.2 upgrade tests

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • None
    • system tests
    • None

    Description

      The 0.10.2 system test failed in one of the upgrade tests from 0.8.2.2:
      http://testing.confluent.io/confluent-kafka-0-10-2-system-test-results/?prefix=2017-03-21--001.1490092219--apache--0.10.2--4a019bd/TestUpgrade/test_upgrade/from_kafka_version=0.8.2.2.to_message_format_version=None.compression_types=.snappy.new_consumer=False/

      [INFO  - 2017-03-21 07:35:48,802 - runner_client - log - lineno:221]: RunnerClient: kafkatest.tests.core.upgrade_test.TestUpgrade.test_upgrade.from_kafka_version=0.8.2.2.to_message_format_version=None.compression_types=.snappy.new_consumer=False: FAIL: Kafka server didn't finish startup
      Traceback (most recent call last):
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/venv/local/lib/python2.7/site-packages/ducktape-0.6.0-py2.7.egg/ducktape/tests/runner_client.py", line 123, in run
          data = self.run_test()
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/venv/local/lib/python2.7/site-packages/ducktape-0.6.0-py2.7.egg/ducktape/tests/runner_client.py", line 176, in run_test
          return self.test_context.function(self.test)
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/venv/local/lib/python2.7/site-packages/ducktape-0.6.0-py2.7.egg/ducktape/mark/_mark.py", line 321, in wrapper
          return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/tests/kafkatest/tests/core/upgrade_test.py", line 125, in test_upgrade
          self.run_produce_consume_validate(core_test_action=lambda: self.perform_upgrade(from_kafka_version,
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/tests/kafkatest/tests/produce_consume_validate.py", line 114, in run_produce_consume_validate
          core_test_action(*args)
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/tests/kafkatest/tests/core/upgrade_test.py", line 126, in <lambda>
          to_message_format_version))
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/tests/kafkatest/tests/core/upgrade_test.py", line 52, in perform_upgrade
          self.kafka.start_node(node)
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/tests/kafkatest/services/kafka/kafka.py", line 222, in start_node
          monitor.wait_until("Kafka Server.*started", timeout_sec=30, backoff_sec=.25, err_msg="Kafka server didn't finish startup")
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/venv/local/lib/python2.7/site-packages/ducktape-0.6.0-py2.7.egg/ducktape/cluster/remoteaccount.py", line 642, in wait_until
          return wait_until(lambda: self.acct.ssh("tail -c +%d %s | grep '%s'" % (self.offset+1, self.log, pattern), allow_fail=True) == 0, **kwargs)
        File "/var/lib/jenkins/workspace/system-test-kafka-0.10.2/kafka/venv/local/lib/python2.7/site-packages/ducktape-0.6.0-py2.7.egg/ducktape/utils/util.py", line 36, in wait_until
      

      Logs:

      ==> ./KafkaService-0-140646398705744/worker9/server-start-stdout-stderr.log <==
      [2017-03-21 07:35:18,250] DEBUG Leaving process event (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] INFO EventThread shut down (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,250] INFO [Kafka Server 2], shut down completed (kafka.server.KafkaServer)
      Error: Exception thrown by the agent : java.rmi.server.ExportException: Port already in use: 9192; nested exception is: 
      	java.net.BindException: Address already in use
      

      That's from starting the upgraded broker, which seems to indicate that 0.8.2.2 was not properly shut down or has its RMI port in the close-wait state.

      Since there probably isn't much to do about 0.8.2.2 the test should probably be hardened to either select a random port, or wait for lingering port to become available (can use netstat for that).

      This earlier failrue from the same 0.8.2.2 invocation might be of interest:

      [2017-03-21 07:35:18,233] DEBUG Writing clean shutdown marker at /mnt/kafka-data-logs (kafka.log.LogManager)
      [2017-03-21 07:35:18,235] INFO Shutdown complete. (kafka.log.LogManager)
      [2017-03-21 07:35:18,238] DEBUG Shutting down task scheduler. (kafka.utils.KafkaScheduler)
      [2017-03-21 07:35:18,243] WARN sleep interrupted (kafka.utils.Utils$)
      java.lang.InterruptedException: sleep interrupted
              at java.lang.Thread.sleep(Native Method)
              at kafka.controller.RequestSendThread$$anonfun$liftedTree1$1$1.apply$mcV$sp(ControllerChannelManager.scala:144)
              at kafka.utils.Utils$.swallow(Utils.scala:172)
              at kafka.utils.Logging$class.swallowWarn(Logging.scala:92)
              at kafka.utils.Utils$.swallowWarn(Utils.scala:45)
              at kafka.utils.Logging$class.swallow(Logging.scala:94)
              at kafka.utils.Utils$.swallow(Utils.scala:45)
              at kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:144)
              at kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:131)
              at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
      [2017-03-21 07:35:18,247] DEBUG Closing ZkClient... (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,247] INFO Terminate ZkClient event thread. (org.I0Itec.zkclient.ZkEventThread)
      [2017-03-21 07:35:18,247] DEBUG Closing ZooKeeper connected to worker5:2181 (org.I0Itec.zkclient.ZkConnection)
      [2017-03-21 07:35:18,247] DEBUG Closing session: 0x15aefca53090001 (org.apache.zookeeper.ZooKeeper)
      [2017-03-21 07:35:18,247] DEBUG Closing client for session: 0x15aefca53090001 (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] DEBUG Got notification sessionid:0x15aefca53090001 (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] DEBUG Got WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/brokers/ids for sessionid 0x15aefca53090001 (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] DEBUG Received event: WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/brokers/ids (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,249] DEBUG Got notification sessionid:0x15aefca53090001 (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] DEBUG Got WatchedEvent state:SyncConnected type:NodeDeleted path:/controller for sessionid 0x15aefca53090001 (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] DEBUG Reading reply sessionid:0x15aefca53090001, packet:: clientPath:null serverPath:null finished:false header:: 61,-11  replyHeader:: 61,81,0  request:: null response:: null (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] DEBUG Disconnecting client for session: 0x15aefca53090001 (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,249] INFO Session: 0x15aefca53090001 closed (org.apache.zookeeper.ZooKeeper)
      [2017-03-21 07:35:18,249] DEBUG Closing ZkClient...done (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] DEBUG ignoring event '{NodeChildrenChanged | /brokers/ids}' since shutdown triggered (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] DEBUG An exception was thrown while closing send thread for session 0x15aefca53090001 : Unable to read additional data from server sessionid 0x15aefca53090001, likely server has closed socket (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,250] DEBUG Leaving process event (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] DEBUG Received event: WatchedEvent state:SyncConnected type:NodeDeleted path:/controller (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] DEBUG ignoring event '{NodeDeleted | /controller}' since shutdown triggered (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] DEBUG Leaving process event (org.I0Itec.zkclient.ZkClient)
      [2017-03-21 07:35:18,250] INFO EventThread shut down (org.apache.zookeeper.ClientCnxn)
      [2017-03-21 07:35:18,250] INFO [Kafka Server 2], shut down completed (kafka.server.KafkaServer)
      Error: Exception thrown by the agent : java.rmi.server.ExportException: Port already in use: 9192; nested exception is: 
              java.net.BindException: Address already in use
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            edenhill Magnus Edenhill
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: