Versions Compared

Key

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

Table of Contents

This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state[One of "Under Discussion", "Accepted", "Rejected"]

Discussion thread: Nonehere [Change the link from the KIP proposal email archive to your own email thread]

JIRA: Nonehere [Change the link from KAFKA-1 to your own ticket]

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

...

Describe the problems you are trying to solve.

Public Interfaces

Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.

A public interface is any change to the following:

...

Binary log format

...

The network protocol and api behavior

At Intuit, we read messages of Kafka and decrypt using an in-house deserializer. This deserializer can throw an exception, either temporary (due to loss of network connection) or permanent (e.g. corrupted data), in which case the KafkaConsumer throws the exception.

Recovery is not straight-forward.

In the permanent case, we just want to skip the message (maybe writing it to a dead-letter queue).

In the temporary case, we might wish to retry the decryption before either giving up, or re-throwing exception.

We want to combine that with support for deserialization to CloudEvents. 

It would be good to establish a pattern for this, rather than each team or organization coming up with their own solutions.

  • The community benefits from having gotten it right, once, one time.
  • Everyone understands how they work.
  • Well thought out by the Kafka team.
  • No proprietary solutions.

Public Interfaces

...

  • org/apache/kafka/common/serialization/ExceptionHandlingDeserializer
  • org/apache/kafka/common/serialization/RetryDeserializer
  • org/apache/kafka/common/errorsserialization/PipeDeserializer
  • org/apache/kafka/clients/producer
  • org/apache/kafka/clients/consumer (eventually, once stable)

  • Monitoring

  • Command line tools and arguments

  • Anything else that will likely break existing users in some way when they upgrade

Proposed Changes

  • common/serialization/PipeSerializer

Proposed Changes

I propose to introduce three new deserializers and one new serializer.

They all follow the decorator pattern. 

ExceptionHandlingDeserializer

This class wraps a delegate deserializer specified by a property:

exception.handling.deserializer.delegate=org.apache.kafka.common.deserialization.ListDeserializer

To deserialize messages, it calls the delegate. If the delegate throws an exception, it return an object contaiing the exception, if there is no exception, then it simple returns the result.

We only want to support some exception (the permanent ones), so we need a matching strategy.

exception.handling.deserializer.matcher=org.apache.kafka.common.deserialization.ExceptionHandlingDeserializer.MatchAllExceptions

The matcher needs to implement a functional interface:

@FunctionalInterface
public interface Matcher {
boolean matches(Exception e);
}

Proof of concept PR

RetryDeserializer 

Again, this wraps a delegate deserializer specified by a property:

retry.deserializer.delegate=org.apache.kafka.common.deserialization.ListDeserializer
retry.deserializer.backoff=org.apache.kafka.common.deserialization.RetryDeserializer.ExponentialBackoff
# https://docs.aws.amazon.com/step-functions/latest/dg/concepts-error-handling.html#error-handling-retrying-after-an-error
retry.deserializer.backoff.exponential.intervalSeconds=1
retry.deserializer.backoff.exponential.maxAttempts=5
retry.deserializer.backoff.exponential.backoffRate=1.2

The backoff policy is configurable and pluggable. One strategy would be no back-off attempts. No back-off would be suitable for exceptions that are permanent.

PipeDeserializer/PipeSerializer

The above deserializer need to be composed together to create more complex serialization and deserialization. For example,

  1. Decrypt message into bytes
  2. Wrap that decryption in retry.
  3. Wrap the retry in exception handling. 
  4. Convert the result into an object (e.g. a CloudEvent using CloudEventsDeserializer).

Putting that all together:

key.deseriailzer=org.apache.kafka.common.deserialization.ExceptionHandlingDeserializer 
exception.handling.deserializer.delegate=org.apache.kafka.common.deserialization.RetryDeserializer
retry.deseriailzer.delegate=org.apache.kafka.common.deserialization.PipeDeserializer
pipe.deserializer.from=com.example.DecryptionDeserializer
pipe.deserializer.to=io.cloudevents.kafka.CloudEventsDeserializer


Proof of concept PRDescribe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?
  • If we are changing behavior how will we phase out the older behavior?
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

Test Plan

Describe in few sentences how the KIP will be tested. We are mostly interested in system tests (since unit-tests are specific to implementation details). How will we know that the implementation works as expected? How will we know nothing broke?

Rejected Alternatives

  • New deserializers that support many common patterns.
  • Well understood

Test Plan

These can all be unit tested. They also need to be battle-tested with users.

Rejected Alternatives

Anyone can build themself in-house monolithic deserializer, but that's inflexible and results in deserializer explosion, ultimately you end up with CloudEventsDecryptingRetryExceptionHandlingDeserializier. I hope I don't need to explaining. LMK if you don't understand why that is undesirable. If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.