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

Document that stores must not be closed when Processors are closed

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.10.2.0
    • 0.10.2.1, 0.11.0.0
    • streams
    • None

    Description

      I have a streams job, that previously worked, that consumes and writes to a large number of topics with many partitions and that uses many threads. I upgraded the job to 0.10.2.0. The job now fails after a short time running, seemingly after a rebalance.

      WARN 2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:160] : Unexpected state transition from RUNNING to NOT_RUNNING

      The first observation is that Streams is no longer outputting exceptions and backtraces. I had to add code to get this information.

      The exception:

      Exception: org.apache.kafka.streams.errors.StreamsException: Exception caught in process. taskId=1_225, processor=KSTREAM-SOURCE-0000000003, topic=some_topic, partition=225, offset=266411
      org.apache.kafka.streams.errors.StreamsException: Exception caught in process. taskId=1_225, processor=KSTREAM-SOURCE-0000000003, topic=some_topic, partition=225, offset=266411
      at org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:216)
      at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:641)
      at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:368)
      Caused by: org.apache.kafka.streams.errors.InvalidStateStoreException: Store someStore-201701060400 is currently closed
      at org.apache.kafka.streams.state.internals.RocksDBStore.validateStoreOpen(RocksDBStore.java:205)
      at org.apache.kafka.streams.state.internals.RocksDBStore.put(RocksDBStore.java:221)
      at org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.put(RocksDBSegmentedBytesStore.java:74)
      at org.apache.kafka.streams.state.internals.ChangeLoggingSegmentedBytesStore.put(ChangeLoggingSegmentedBytesStore.java:54)
      at org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore.put(MeteredSegmentedBytesStore.java:101)
      at org.apache.kafka.streams.state.internals.RocksDBWindowStore.put(RocksDBWindowStore.java:109)
      ... X more

      The error occurs for many partitions.

      This was preceded by (for this partition):

      INFO 2017-03-19 18:03:16,403 [StreamThread-10][ConsumerCoordinator.java:393] : Revoking previously assigned partitions [some_topic-225] for group some_job
      INFO 2017-03-19 18:03:16,403 [StreamThread-10][StreamThread.java:254] : stream-thread [StreamThread-10] partitions [[some_topic-225]] revoked at the beginning of consumer rebalance.
      INFO 2017-03-19 18:03:16,403 [StreamThread-10][StreamThread.java:1056] : stream-thread [StreamThread-10] Closing a task's topology 1_225
      INFO 2017-03-19 18:03:17,887 [StreamThread-10][StreamThread.java:544] : stream-thread [StreamThread-10] Flushing state stores of task 1_225
      INFO 2017-03-19 18:03:17,887 [StreamThread-10][StreamThread.java:534] : stream-thread [StreamThread-10] Committing consumer offsets of task 1_225
      INFO 2017-03-19 18:03:17,891 [StreamThread-10][StreamThread.java:1012] : stream-thread [StreamThread-10] Updating suspended tasks to contain active tasks [[1_225, 0_445, 0_30]]
      INFO 2017-03-19 18:03:17,891 [StreamThread-10][StreamThread.java:1019] : stream-thread [StreamThread-10] Removing all active tasks [[1_225, 0_445, 0_30]]

      INFO 2017-03-19 18:03:19,925 [StreamThread-10][ConsumerCoordinator.java:252] : Setting newly assigned partitions [some_tpoic-225] for group some_job
      INFO 2017-03-19 18:03:19,927 [StreamThread-10][StreamThread.java:228] : stream-thread [StreamThread-10] New partitions [[some_topic-225]] assigned at the end of consumer rebalance.
      INFO 2017-03-19 18:03:19,929 [StreamThread-10][StreamTask.java:333] : task [1_225] Initializing processor nodes of the topology

      Something happens. What ???

      INFO 2017-03-19 18:03:20,135 [StreamThread-10][StreamThread.java:1045] : stream-thread [StreamThread-10] Closing a task 1_225
      INFO 2017-03-19 18:03:20,355 [StreamThread-10][StreamThread.java:544] : stream-thread [StreamThread-10] Flushing state stores of task 1_225
      INFO 2017-03-19 18:03:20,355 [StreamThread-10][StreamThread.java:523] : stream-thread [StreamThread-10] Closing the state manager of task 1_225
      INFO 2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:1019] : stream-thread [StreamThread-10] Removing all active tasks [[1_225, 0_445, 0_30]]
      INFO 2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:1034] : stream-thread [StreamThread-10] Removing all standby tasks [[]]
      INFO 2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:427] : stream-thread [StreamThread-10] Stream thread shutdown complete

      Attachments

        Issue Links

          Activity

            People

              damianguy Damian Guy
              elevy Elias Levy
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: