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

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

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

            damianguy Damian Guy added a comment -

            Looks like the issue is in ProcessorStateManager. We only restore and hence assign partitions if the store is persistent

            if (store.persistent()) {
                            log.trace("Preparing standby replica of persistent state store {} with changelog topic {}", store.name(), topic);
            
                            restoreCallbacks.put(topic, stateRestoreCallback);
                        }
            
            damianguy Damian Guy added a comment - Looks like the issue is in ProcessorStateManager . We only restore and hence assign partitions if the store is persistent if (store.persistent()) { log.trace( "Preparing standby replica of persistent state store {} with changelog topic {}" , store.name(), topic); restoreCallbacks.put(topic, stateRestoreCallback); }
            farmdawgnation Matt Farmer added a comment -

            Is the answer here, then, to just remove that if statement?

            farmdawgnation Matt Farmer added a comment - Is the answer here, then, to just remove that if statement?
            githubbot ASF GitHub Bot added a comment -

            GitHub user dguy opened a pull request:

            https://github.com/apache/kafka/pull/4239

            KAFKA-6214: enable use of in-memory store for standby tasks

            Remove the flag in `ProcessorStateManager` that checks if a store is persistent when registering it as a standby task.
            Updated the smoke test to use an in-memory store.

                1. Committer Checklist (excluded from commit message)
            • [ ] Verify design and implementation
            • [ ] Verify test coverage and CI build status
            • [ ] Verify documentation (including upgrade notes)

            You can merge this pull request into a Git repository by running:

            $ git pull https://github.com/dguy/kafka kafka-6214

            Alternatively you can review and apply these changes as the patch at:

            https://github.com/apache/kafka/pull/4239.patch

            To close this pull request, make a commit to your master/trunk branch
            with (at least) the following in the commit message:

            This closes #4239


            commit a9cc7b3bbf2bc42462d0876c6c13fdc8173f47ec
            Author: Damian Guy <damian.guy@gmail.com>
            Date: 2017-11-20T13:30:35Z

            enable use of in-memory store for standby tasks


            githubbot ASF GitHub Bot added a comment - GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/4239 KAFKA-6214 : enable use of in-memory store for standby tasks Remove the flag in `ProcessorStateManager` that checks if a store is persistent when registering it as a standby task. Updated the smoke test to use an in-memory store. Committer Checklist (excluded from commit message) [ ] Verify design and implementation [ ] Verify test coverage and CI build status [ ] Verify documentation (including upgrade notes) You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka kafka-6214 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/4239.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #4239 commit a9cc7b3bbf2bc42462d0876c6c13fdc8173f47ec Author: Damian Guy <damian.guy@gmail.com> Date: 2017-11-20T13:30:35Z enable use of in-memory store for standby tasks
            guozhang Guozhang Wang added a comment -

            Issue resolved by pull request 4239
            https://github.com/apache/kafka/pull/4239

            guozhang Guozhang Wang added a comment - Issue resolved by pull request 4239 https://github.com/apache/kafka/pull/4239
            githubbot ASF GitHub Bot added a comment -

            Github user asfgit closed the pull request at:

            https://github.com/apache/kafka/pull/4239

            githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4239

            People

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

              Dates

                Created:
                Updated:
                Resolved: