Child pages
  • KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user
Skip to end of metadata
Go to start of metadata

Status

Current state: "Under Discussion"

Discussion thread: here

JIRA KAFKA-4808 - send of null key to a compacted topic should throw error back to user Open

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

Motivation

Currently when a user sends a null key to a log compacted topic, the broker returns CorruptRecordException, which is a retriable exception. As such, the producer keeps retrying until retries are exhausted or request.timeout.ms expires and eventually throws a TimeoutException. This is confusing and not user-friendly. 

The broker should throw a non-retriable exception with a message explaining the error.

New or Changed Public Interfaces

This KIP introduces a new error type INVALID_KEY.

It introduces a new non-retriable exception InvalidRecordKeyException. Going forward user request will fail without doing retries, if they try to produce to a log compacted topic with a null key.

Proposed Changes

Add a new error code and exception as follows :

Error Code :

INVALID_RECORD_KEY(45, new InvalidRecordKeyException("Message key is invalid."))

Exception :

Compatibility, Deprecation, and Migration Plan

We will only return this exception for ProduceRequest V3 or higher. The older version of ProduceRequest will still get the CorruptRecordException.

Rejected Alternatives

  • No labels