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

Using standby replicas with an in memory state store causes Streams to crash

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.11.0.1
    • 1.0.1, 1.1.0
    • streams

    Description

      We decided to start experimenting with Standby Replicas of our State Stores by setting the following configuration setting:

      num.standby.replicas=1
      

      Most applications did okay with this except for one that used an in memory state store instead of a persistent state store. With the new configuration, the first instance of this application booted fine. When the second instance came up, both instances crashed with the following exception:

      java.lang.IllegalStateException: Consumer is not subscribed to any topics or assigned any partitions
              at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1037)
              at org.apache.kafka.streams.processor.internals.StreamThread.maybeUpdateStandbyTasks(StreamThread.java:752)
              at org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:524)
              at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:480)
              at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:457)
      

      Monit attempted to restart both instances but they would just continue to crash over and over again. The state store in our problematic application is declared like so:

      Stores
          .create("TheStateStore")
          .withStringKeys()
          .withStringValues()
          .inMemory()
          .build()
      

      Luckily we had a config switch in place that could turn on an alternate, persistent state store. As soon as we flipped to the persistent state store, things started working as we expected.

      Attachments

        Issue Links

          Activity

            People

              damianguy Damian Guy
              farmdawgnation Matt Farmer
              Votes:
              1 Vote for this issue
              Watchers:
              9 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: