Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

A Kafka Streams client need to handle multiple different types of exceptions. We try to summarize what kind of exceptions are there, and how Kafka Streams should handle those. In general, Kafka Streams should be resilient to exceptions and keep processing even if some internal exceptions occur.

Types of Exceptions:

 

First, we need to distingues between retryable and non-retryable retriable (ie, fatal) exceptions. For non-retryable/fatal exceptions, Kafka Streams is doomed to fail and cannot start/continue to process data.

 

 

the exception thrown during processing of this record. Null if no error occurred.
     *                  Possible thrown exceptions include:
     *
     *                  Non-Retriable exceptions (fatal, the message will never be sent):
     *
     *                  InvalidTopicException
     *                  OffsetMetadataTooLargeException
     *                  RecordBatchTooLargeException
     *                  RecordTooLargeException
     *                  UnknownServerException
     *
     *                  Retriable exceptions (transient, may be covered by increasing #.retries):
     *
     *                  CorruptRecordException
     *                  OffsetMetadataTooLargeException
     *                  NotEnoughReplicasAfterAppendException
     *                  NotEnoughReplicasException
     *                  OffsetOutOfRangeException
     *                  TimeoutException
     *                  UnknownTopicOrPartitionException

...

  • Consumer exceptions:
    • expected:
      • ConfigException (fatal)
      • InvalidOffsetException (handled by StreamThread)
        1. OffsetOutOfRangeException
        2. NoOffsetForPartitionsException
      • CommitFailedException [non-EOS only] (handled by StreamThread: swallow and retry on next commit) -> not swallowed for RebalanceCallback (need to get fixed)
      • QuotaViolationException (fatal ?)
      • AuthorizationException (fatal)
      • SecurityDisabledException (fatal)
      • InvalidTopicException (fatal)
      • all RetryableException
    • should never happen (all fatal):
      • ConcurrentModificationException
      • WakeupException
      • InterruptedException
      • IllegalArgumentException
      • IllegalStateException
      • All ApiException that are not mentioned somewhere else
  • Producer exception:
    • expected:
      • BufferExhausedException (fatal)
      • SerializationException (fatal)
      • ProducerFencedException
      • SecurityDisabledException (fatal)
      • all RetryableException
    • should never happen:
      • All ApiException that are not mentioned somewhere else
  • AdminClient exceptions:
  • State store exceptions:
  • Serialization exceptions:
  • StreamsException
  • User-code exceptions:

...

Gliffy Diagram
nameExceptionHandling

 

Exception progagation / chain-reactions

For non-retryable/fatal exceptions there are two different subtypes. For the first type, we expect that it will eventually