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

Consumer.poll() stuck in loop on non-existent topic manually assigned

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • None
    • None
    • clients
    • None

    Description

      The behavior of a consumer on poll() for a non-existing topic is surprisingly different/inconsistent
      between a consumer that subscribed to the topic and one that had the topic-partition manually assigned.

      The "subscribed" consumer will return an empty collection
      The "assigned" consumer will loop forever - this feels a bug to me.

      sample snippet to reproduce:

      KafkaConsumer<String, String> assignKc = new KafkaConsumer<>(props1);
      KafkaConsumer<String, String> subsKc = new KafkaConsumer<>(props2);
      List<TopicPartition> tps = new ArrayList<>();
      tps.add(new TopicPartition("topic-not-exists", 0));
      assignKc.assign(tps);

      subsKc.subscribe(Arrays.asList("topic-not-exists"));

      System.out.println("********* subscribe k consumer ");
      ConsumerRecords<String, String> crs2 = subsKc.poll(1000L);
      print("subscribeKc", crs2); // returns empty

      System.out.println("********* assign k consumer ");
      ConsumerRecords<String, String> crs1 = assignKc.poll(1000L);
      // will loop forever !
      print("assignKc", crs1);

      the logs for the "assigned" consumer show:
      [2016-05-18 17:33:09,907] DEBUG Updated cluster metadata version 8 to Cluster(nodes = [192.168.10.18:9093 (id: 0 rack: null)], partitions = []) (org.apache.kafka.clients.Metadata)
      [2016-05-18 17:33:09,908] DEBUG Partition topic-not-exists-0 is unknown for fetching offset, wait for metadata refresh (org.apache.kafka.clients.consumer.internals.Fetcher)
      [2016-05-18 17:33:10,010] DEBUG Sending metadata request

      {topics=[topic-not-exists]}

      to node 0 (org.apache.kafka.clients.NetworkClient)
      [2016-05-18 17:33:10,011] WARN Error while fetching metadata with correlation id 9 :

      {topic-not-exists=UNKNOWN_TOPIC_OR_PARTITION}

      (org.apache.kafka.clients.NetworkClient)

      Attachments

        Issue Links

          Activity

            People

              ecomar Edoardo Comar
              ecomar Edoardo Comar
              Votes:
              1 Vote for this issue
              Watchers:
              13 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: